Skip to content

Commit

Permalink
PR comments
Browse files Browse the repository at this point in the history
  • Loading branch information
pranavrth committed Oct 11, 2023
1 parent 269f8d7 commit cd0009c
Show file tree
Hide file tree
Showing 6 changed files with 21 additions and 22 deletions.
4 changes: 2 additions & 2 deletions examples/describe_topics.c
Original file line number Diff line number Diff line change
Expand Up @@ -204,8 +204,8 @@ static void print_topic_info(const rd_kafka_TopicDescription_t *topic) {
size_t authorized_operations_cnt;
const rd_kafka_TopicPartitionInfo_t **partitions;
size_t partition_cnt;
rd_kafka_Uuid_t *topic_id = rd_kafka_TopicDescription_topic_id(topic);
char *topic_id_str = rd_kafka_Uuid_base64str(topic_id);
const rd_kafka_Uuid_t *topic_id = rd_kafka_TopicDescription_topic_id(topic);
const char *topic_id_str = rd_kafka_Uuid_base64str(topic_id);

if (rd_kafka_error_code(error)) {
printf("Topic: %s (Topic Id: %s) has error[%" PRId32 "]: %s\n",
Expand Down
10 changes: 5 additions & 5 deletions src/rdkafka.c
Original file line number Diff line number Diff line change
Expand Up @@ -5050,7 +5050,7 @@ rd_kafka_Uuid_t *rd_kafka_Uuid_new(int64_t most_significant_bits,
*
* @remark Dynamically allocated. Deallocate (free) after use.
*/
rd_kafka_Uuid_t *rd_kafka_Uuid_copy(rd_kafka_Uuid_t *uuid) {
rd_kafka_Uuid_t *rd_kafka_Uuid_copy(const rd_kafka_Uuid_t *uuid) {
rd_kafka_Uuid_t *copy_uuid = rd_kafka_Uuid_new(
uuid->most_significant_bits, uuid->least_significant_bits);
if (*uuid->base64str)
Expand All @@ -5067,7 +5067,7 @@ void rd_kafka_Uuid_destroy(rd_kafka_Uuid_t *uuid) {
rd_free(uuid);
}

char *rd_kafka_Uuid_base64str(rd_kafka_Uuid_t *uuid) {
const char *rd_kafka_Uuid_base64str(const rd_kafka_Uuid_t *uuid) {
if (*uuid->base64str)
return uuid->base64str;

Expand All @@ -5087,17 +5087,17 @@ char *rd_kafka_Uuid_base64str(rd_kafka_Uuid_t *uuid) {
if (!out_base64_str)
return NULL;

rd_strlcpy(uuid->base64str, out_base64_str,
rd_strlcpy((char *) uuid->base64str, out_base64_str,
23 /* Removing extra ('=') padding */);
rd_free(out_base64_str);
return uuid->base64str;
}

int64_t rd_kafka_Uuid_least_significant_bits(rd_kafka_Uuid_t *uuid) {
int64_t rd_kafka_Uuid_least_significant_bits(const rd_kafka_Uuid_t *uuid) {
return uuid->least_significant_bits;
}


int64_t rd_kafka_Uuid_most_significant_bits(rd_kafka_Uuid_t *uuid) {
int64_t rd_kafka_Uuid_most_significant_bits(const rd_kafka_Uuid_t *uuid) {
return uuid->most_significant_bits;
}
10 changes: 5 additions & 5 deletions src/rdkafka.h
Original file line number Diff line number Diff line change
Expand Up @@ -1645,7 +1645,7 @@ rd_kafka_message_leader_epoch(const rd_kafka_message_t *rkmessage);
* @return base64 encoded string for the given UUID or NULL in case of some
* issue with the conversion or the conversion is not supported.
*/
RD_EXPORT char *rd_kafka_Uuid_base64str(rd_kafka_Uuid_t *uuid);
RD_EXPORT const char *rd_kafka_Uuid_base64str(const rd_kafka_Uuid_t *uuid);

/**
* @brief Gets least significant 64 bits for the given UUID.
Expand All @@ -1654,7 +1654,7 @@ RD_EXPORT char *rd_kafka_Uuid_base64str(rd_kafka_Uuid_t *uuid);
*
* @return least significant 64 bits for the given UUID.
*/
RD_EXPORT int64_t rd_kafka_Uuid_least_significant_bits(rd_kafka_Uuid_t *uuid);
RD_EXPORT int64_t rd_kafka_Uuid_least_significant_bits(const rd_kafka_Uuid_t *uuid);


/**
Expand All @@ -1664,7 +1664,7 @@ RD_EXPORT int64_t rd_kafka_Uuid_least_significant_bits(rd_kafka_Uuid_t *uuid);
*
* @return most significant 64 bits for the given UUID.
*/
RD_EXPORT int64_t rd_kafka_Uuid_most_significant_bits(rd_kafka_Uuid_t *uuid);
RD_EXPORT int64_t rd_kafka_Uuid_most_significant_bits(const rd_kafka_Uuid_t *uuid);


/**
Expand All @@ -1687,7 +1687,7 @@ RD_EXPORT rd_kafka_Uuid_t *rd_kafka_Uuid_new(int64_t most_significant_bits,
* @return A newly allocated copy of the provided UUID.
* @remark Must be freed after use using rd_kafka_Uuid_destroy()
*/
RD_EXPORT rd_kafka_Uuid_t *rd_kafka_Uuid_copy(rd_kafka_Uuid_t *uuid);
RD_EXPORT rd_kafka_Uuid_t *rd_kafka_Uuid_copy(const rd_kafka_Uuid_t *uuid);

/**
* @brief Destroy the provided uuid.
Expand Down Expand Up @@ -8249,7 +8249,7 @@ rd_kafka_TopicDescription_name(const rd_kafka_TopicDescription_t *topicdesc);
* @remark The lifetime of the returned memory is the same
* as the lifetime of the \p topicdesc object.
*/
RD_EXPORT rd_kafka_Uuid_t *rd_kafka_TopicDescription_topic_id(
RD_EXPORT const rd_kafka_Uuid_t *rd_kafka_TopicDescription_topic_id(
const rd_kafka_TopicDescription_t *topicdesc);

/**
Expand Down
15 changes: 7 additions & 8 deletions src/rdkafka_admin.c
Original file line number Diff line number Diff line change
Expand Up @@ -8020,7 +8020,7 @@ rd_kafka_TopicPartitionInfo_destroy(rd_kafka_TopicPartitionInfo_t *pinfo) {
*/
static rd_kafka_TopicDescription_t *rd_kafka_TopicDescription_new(
const char *topic,
rd_kafka_Uuid_t *topic_id,
rd_kafka_Uuid_t topic_id,
const struct rd_kafka_metadata_partition *partitions,
int partition_cnt,
const struct rd_kafka_metadata_broker *brokers_sorted,
Expand All @@ -8034,7 +8034,7 @@ static rd_kafka_TopicDescription_t *rd_kafka_TopicDescription_new(
int i;
topicdesc = rd_calloc(1, sizeof(*topicdesc));
topicdesc->topic = rd_strdup(topic);
topicdesc->topic_id = rd_kafka_Uuid_copy(topic_id);
topicdesc->topic_id = topic_id;
topicdesc->partition_cnt = partition_cnt;
topicdesc->is_internal = is_internal;
if (error)
Expand Down Expand Up @@ -8066,7 +8066,7 @@ static rd_kafka_TopicDescription_t *rd_kafka_TopicDescription_new(
*/
static rd_kafka_TopicDescription_t *
rd_kafka_TopicDescription_new_error(const char *topic,
rd_kafka_Uuid_t *topic_id,
rd_kafka_Uuid_t topic_id,
rd_kafka_error_t *error) {
return rd_kafka_TopicDescription_new(topic, topic_id, NULL, 0, NULL,
NULL, 0, NULL, 0, rd_false, error);
Expand All @@ -8079,7 +8079,6 @@ rd_kafka_TopicDescription_destroy(rd_kafka_TopicDescription_t *topicdesc) {
RD_IF_FREE(topicdesc->topic, rd_free);
RD_IF_FREE(topicdesc->error, rd_kafka_error_destroy);
RD_IF_FREE(topicdesc->authorized_operations, rd_free);
RD_IF_FREE(topicdesc->topic_id, rd_kafka_Uuid_destroy);
for (i = 0; i < topicdesc->partition_cnt; i++)
rd_kafka_TopicPartitionInfo_destroy(topicdesc->partitions[i]);
rd_free(topicdesc->partitions);
Expand Down Expand Up @@ -8146,9 +8145,9 @@ rd_kafka_TopicDescription_error(const rd_kafka_TopicDescription_t *topicdesc) {
return topicdesc->error;
}

rd_kafka_Uuid_t *rd_kafka_TopicDescription_topic_id(
const rd_kafka_Uuid_t *rd_kafka_TopicDescription_topic_id(
const rd_kafka_TopicDescription_t *topicdesc) {
return topicdesc->topic_id;
return &topicdesc->topic_id;
}

const rd_kafka_TopicDescription_t **rd_kafka_DescribeTopics_result_topics(
Expand Down Expand Up @@ -8249,7 +8248,7 @@ rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req,
mdi->topics[i].topic_authorized_operations,
&authorized_operation_cnt);
topicdesc = rd_kafka_TopicDescription_new(
md->topics[i].topic, &mdi->topics[i].topic_id,
md->topics[i].topic, mdi->topics[i].topic_id,
md->topics[i].partitions,
md->topics[i].partition_cnt, mdi->brokers_sorted,
mdi->brokers, md->broker_cnt, authorized_operations,
Expand All @@ -8261,7 +8260,7 @@ rd_kafka_DescribeTopicsResponse_parse(rd_kafka_op_t *rko_req,
md->topics[i].err, "%s",
rd_kafka_err2str(md->topics[i].err));
topicdesc = rd_kafka_TopicDescription_new_error(
md->topics[i].topic, &mdi->topics[i].topic_id,
md->topics[i].topic, mdi->topics[i].topic_id,
error);
rd_kafka_error_destroy(error);
}
Expand Down
2 changes: 1 addition & 1 deletion src/rdkafka_admin.h
Original file line number Diff line number Diff line change
Expand Up @@ -526,7 +526,7 @@ struct rd_kafka_TopicPartitionInfo_s {
*/
struct rd_kafka_TopicDescription_s {
char *topic; /**< Topic name */
rd_kafka_Uuid_t *topic_id; /**< Topic Id */
rd_kafka_Uuid_t topic_id; /**< Topic Id */
int partition_cnt; /**< Number of partitions in \p partitions*/
rd_bool_t is_internal; /**< Is the topic is internal to Kafka? */
rd_kafka_TopicPartitionInfo_t **partitions; /**< Partitions */
Expand Down
2 changes: 1 addition & 1 deletion tests/0081-admin.c
Original file line number Diff line number Diff line change
Expand Up @@ -3169,7 +3169,7 @@ static void do_test_DescribeTopics(const char *what,
const rd_kafka_DescribeTopics_result_t *res;
const rd_kafka_TopicDescription_t **result_topics;
const rd_kafka_TopicPartitionInfo_t **partitions;
rd_kafka_Uuid_t *topic_id;
const rd_kafka_Uuid_t *topic_id;
size_t partitions_cnt;
size_t result_topics_cnt;
char errstr[128];
Expand Down

0 comments on commit cd0009c

Please sign in to comment.