Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KIP 951 - All phases - cherry picked #4756

Merged
merged 19 commits into from
Jun 17, 2024
Merged

KIP 951 - All phases - cherry picked #4756

merged 19 commits into from
Jun 17, 2024

Conversation

anchitj
Copy link
Member

@anchitj anchitj commented Jun 13, 2024

No description provided.

@anchitj anchitj requested a review from a team as a code owner June 13, 2024 17:32
@anchitj anchitj force-pushed the dev_951_cherry_pick branch 6 times, most recently from 89ef329 to 98315bc Compare June 14, 2024 09:06
@emasab
Copy link
Collaborator

emasab commented Jun 14, 2024

Please add this fix too to the PR.
To avoid logging when the id doesn't change and to use rd_kafka_Uuid_base64str instead of rd_kafka_Uuid_str that allocates and leaks memory (and is only for debugging)

                if (rd_kafka_Uuid_cmp(mdit->topic_id, RD_KAFKA_UUID_ZERO) &&
                    rd_kafka_Uuid_cmp(mdit->topic_id, rkt->rkt_topic_id)) {
                        /* FIXME: an offset reset must be triggered.
                         * when rkt_topic_id wasn't zero.
                         * There are no problems
                         * in test 0107_topic_recreate if offsets in new
                         * topic are lower than in previous one,
                         * causing an out of range and an offset reset,
                         * but the rarer case where they're higher needs
                         * to be checked. */
                        rd_kafka_dbg(
                            rk, TOPIC | RD_KAFKA_DBG_METADATA, "METADATA",
                            "Topic %s changed id from %s to %s",
                            rkt->rkt_topic->str,
                            rd_kafka_Uuid_base64str(&rkt->rkt_topic_id),
                            rd_kafka_Uuid_base64str(&mdit->topic_id));
                        rkt->rkt_topic_id = mdit->topic_id;
                }

@emasab
Copy link
Collaborator

emasab commented Jun 14, 2024

Please rebase with dependency upgrade commit merged

Copy link
Collaborator

@emasab emasab left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

First comments about the code structure and naming

src/rdkafka.c Outdated Show resolved Hide resolved
src/rdkafka.c Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_request.h Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
@anchitj anchitj force-pushed the dev_951_cherry_pick branch 3 times, most recently from 1902d0c to 680af32 Compare June 14, 2024 11:14
src/rdkafka.c Outdated
@@ -4078,6 +4078,10 @@ rd_kafka_op_res_t rd_kafka_poll_cb(rd_kafka_t *rk,
rd_kafka_purge(rk, rko->rko_u.purge.flags);
break;

case RD_KAFKA_OP_METADATA_UPDATE:
rd_kafka_metadata_update_op(rk, rko->rko_u.metadata.mdi);
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

RD_KAFKA_OP_RES_HANDLED must be returned otherwise it's not destroyed. The decision on the fields to use should be done by the handling code

Suggested change
rd_kafka_metadata_update_op(rk, rko->rko_u.metadata.mdi);
res = rd_kafka_metadata_update_op(rk, rko);

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm returning RD_KAFKA_OP_RES_HANDLED from rd_kafka_metadata_update_op not sure if that's correct

src/rdkafka_request.h Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_op.h Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
*
*/

typedef struct rd_kafkap_Fetch_reply_Partition_s {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
typedef struct rd_kafkap_Fetch_reply_Partition_s {
typedef struct rd_kafkap_Fetch_reply_Partition_s {

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated to rd_kafkap_Fetch_reply_tags_Partition_s

src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
rkmce->rkmce_metadata_internal_topic.partitions[mdp->id]
.leader_epoch = mdp_int->leader_epoch;
rkmce->rkmce_mtopic.partitions[mdp->id].leader =
mdp->leader;
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
mdp->leader;
mdp->leader;
rd_kafka_wrunlock(rk);

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a
rd_kafka_broker_destroy(rkb);
here

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated and added rd_kafka_broker_destroy(rkb); after unlock

&rkmce->rkmce_metadata_internal_topic, rd_true,
rd_true, mdi->brokers, mdi->metadata.broker_cnt,
rd_true);
rd_kafka_topic_metadata_update2(
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This needs to be done once per topic, so you can put it outside the for loop you can have a variable
partition_cache_changes that is incremented for each partition and use it in an if outside the partition loop

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There the rkb is necessary just for logging purposes, you can use rd_kafka_broker_t *rkb = rk->rk_internal_rkb; in the if after the partition loop

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added a counter partition_cache_changes which is incremented for each partition cache change, and if the counter> 0 then the rd_kafka_topic_metadata_update2 is called.

src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Show resolved Hide resolved
rkmce->rkmce_metadata_internal_topic.partitions[mdp->id]
.leader_epoch = mdp_int->leader_epoch;
rkmce->rkmce_mtopic.partitions[mdp->id].leader =
mdp->leader;
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add a
rd_kafka_broker_destroy(rkb);
here

src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
rd_kafka_buf_skip_tags(rkbuf);
/* Topic tags */
rd_kafka_buf_skip_tags(rkbuf);
if (request->rkbuf_reqhdr.ApiVersion >= 10 &&
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here it should just populate the topic tags, then the message isn't sent depending on the error code but whenever we have these tags

Suggested change
if (request->rkbuf_reqhdr.ApiVersion >= 10 &&
if (request->rkbuf_reqhdr.ApiVersion >= 10) {
rd_kafkap_Produce_reply_tags_Topic_t *TopicTags =
&ProduceTags.Topic;
rd_kafkap_Produce_reply_tags_Partition_t *PartitionTags =
&TopicTags->Partition;
/* Partition tags count */
TopicTags->TopicName = RD_KAFKAP_STR_DUP(&TopicName);
PartitionTags->Partition = hdr.Partition;
}

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

All that comes after must be simpler and without these conditions, this way:

        /* Partition tags */
        rd_kafka_buf_read_tags(rkbuf,
                               rd_kafkap_Produce_reply_tags_partition_parse,
                               &ProduceTags, &ProduceTags.Topic.Partition);

        /* Topic tags */
        rd_kafka_buf_skip_tags(rkbuf);

        if (request->rkbuf_reqhdr.ApiVersion >= 1) {
                int32_t Throttle_Time;
                rd_kafka_buf_read_i32(rkbuf, &Throttle_Time);

                rd_kafka_op_throttle_time(rkb, rkb->rkb_rk->rk_rep,
                                          Throttle_Time);
        }

        /* ProduceResponse tags */
        rd_kafka_buf_read_tags(rkbuf, rd_kafkap_Produce_reply_tags_parse,
                               &ProduceTags);

        rd_kafka_handle_Produce_metadata_update(rkb, &ProduceTags);

        rd_kafkap_Produce_reply_tags_destroy(&ProduceTags);
        return hdr.ErrorCode;

err_parse:
        rd_kafkap_Produce_reply_tags_destroy(&ProduceTags);
        return rkbuf->rkbuf_err;
err:
        rd_kafkap_Produce_reply_tags_destroy(&ProduceTags);
        return RD_KAFKA_RESP_ERR__BAD_MSG;

It's the rd_kafka_handle_Produce_metadata_update that sends an update op if the ProduceTags->leader_change_cnt is non zero.
rd_kafkap_Produce_reply_tags_destroy instead frees the TopicName and NodeEndpoints.NodeEndpoints if not NULL

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated. Would do the similar for fetch

Copy link
Collaborator

@emasab emasab left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Comments about creation of the metadata update operation

src/rdkafka_request.c Outdated Show resolved Hide resolved
Copy link
Collaborator

@emasab emasab left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Changes about the fetch tags

src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
@anchitj anchitj force-pushed the dev_951_cherry_pick branch 3 times, most recently from dfc6b36 to ab12eab Compare June 17, 2024 10:49
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_fetcher.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_metadata.c Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_request.h Outdated Show resolved Hide resolved
src/rdkafka_request.c Outdated Show resolved Hide resolved
src/rdkafka_request.c Show resolved Hide resolved
src/rdkafka_request.c Show resolved Hide resolved
Copy link
Collaborator

@emasab emasab left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM! Thanks for the changes

@emasab emasab merged commit f47815b into master Jun 17, 2024
2 checks passed
@emasab emasab deleted the dev_951_cherry_pick branch June 17, 2024 16:50
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants