Skip to content

[KIP-848] Added new error code handling to OffsetCommit and OffsetFetch #4681

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

Merged

Conversation

pranavrth
Copy link
Member

No description provided.

@emasab emasab force-pushed the dev_kip848_use_metadata_cache_and_fixes branch 8 times, most recently from 270f7f7 to 228b93d Compare April 15, 2024 18:02
@pranavrth pranavrth force-pushed the dev_kip848_offset_fetch_commit_error_code_changes_only branch from ff78ef0 to 1d26a60 Compare April 15, 2024 18:17
@@ -230,6 +254,7 @@ static void rd_kafka_assignment_handle_OffsetFetch(rd_kafka_t *rk,
rd_kafka_buf_t *reply,
rd_kafka_buf_t *request,
void *opaque) {

Copy link
Contributor

Choose a reason for hiding this comment

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

Remove this new empty line

offsets->cnt, rk->rk_group_id->str, rd_kafka_err2str(err));
switch (err) {
case RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH:
rd_assert(rk->rk_cgrp->rkcg_group_protocol ==
Copy link
Contributor

Choose a reason for hiding this comment

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

We can remove the assert here too

Comment on lines 3779 to 3833
case RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH:
/* FIXME: Add logs.*/
rd_kafka_cgrp_consumer_expedite_next_heartbeat(rk->rk_cgrp);
break;
Copy link
Contributor

Choose a reason for hiding this comment

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

We can retry the commit in this case, if not a manual commit as the member epoch will be updated and even in case of order change, it has set_offsets set to true, so will take the latest stored offset for given partitions.

Suggested change
case RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH:
/* FIXME: Add logs.*/
rd_kafka_cgrp_consumer_expedite_next_heartbeat(rk->rk_cgrp);
break;
case RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH:
/* FIXME: Add logs.*/
if (!rd_strcmp(rko_orig->rko_u.offset_commit.reason, "manual"))
/* Don't retry manual commits giving this error.
* TODO: do this in a faster and cleaner way
* with a bool. */
break;
if (rd_kafka_cgrp_consumer_defer_offset_commit(
rkcg, rko_orig, rd_kafka_err2str(err)))
return;
break;
/**
 * @brief Defer offset commit (rko) until coordinator is available (KIP-848).
 *
 * @returns 1 if the rko was deferred or 0 if the defer queue is disabled
 *          or rko already deferred.
 */
static int rd_kafka_cgrp_consumer_defer_offset_commit(rd_kafka_cgrp_t *rkcg,
                                                      rd_kafka_op_t *rko,
                                                      const char *reason) {
        /* wait_coord_q is disabled session.timeout.ms after
         * group close() has been initated. */
        if ((rko->rko_u.offset_commit.ts_timeout != 0 &&
             rd_clock() >= rko->rko_u.offset_commit.ts_timeout) ||
            !rd_kafka_q_ready(rkcg->rkcg_wait_coord_q))
                return 0;

        rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "COMMIT",
                     "Group \"%s\": "
                     "unable to OffsetCommit in state %s: %s: "
                     "retrying later",
                     rkcg->rkcg_group_id->str,
                     rd_kafka_cgrp_state_names[rkcg->rkcg_state], reason);

        rko->rko_flags |= RD_KAFKA_OP_F_REPROCESS;

        if (!rko->rko_u.offset_commit.ts_timeout) {
                rko->rko_u.offset_commit.ts_timeout =
                    rd_clock() +
                    (rkcg->rkcg_rk->rk_conf.group_session_timeout_ms * 1000);
        }
        
        /* Reset partition level error before retrying */
        rd_kafka_topic_partition_list_set_err(
            rko->rko_u.offset_commit.partitions,
            RD_KAFKA_RESP_ERR_NO_ERROR);

        rd_kafka_q_enq(rkcg->rkcg_wait_coord_q, rko);

        return 1;
}

@emasab emasab force-pushed the dev_kip848_use_metadata_cache_and_fixes branch from 228b93d to 215ec6e Compare April 17, 2024 06:28
@@ -3692,6 +3815,19 @@ static void rd_kafka_cgrp_op_handle_OffsetCommit(rd_kafka_t *rk,
case RD_KAFKA_RESP_ERR__IN_PROGRESS:
return; /* Retrying */

case RD_KAFKA_RESP_ERR_STALE_MEMBER_EPOCH:
/* FIXME: Add logs.*/
Copy link
Contributor

Choose a reason for hiding this comment

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

There was an expedite HB here

Suggested change
/* FIXME: Add logs.*/
/* FIXME: Add logs.*/
rd_kafka_cgrp_consumer_expedite_next_heartbeat(rk->rk_cgrp);

@pranavrth pranavrth force-pushed the dev_kip848_offset_fetch_commit_error_code_changes_only branch from 36518f2 to 58aca9a Compare April 17, 2024 10:10
Base automatically changed from dev_kip848_use_metadata_cache_and_fixes to dev_kip848 April 17, 2024 11:28
@pranavrth pranavrth force-pushed the dev_kip848_offset_fetch_commit_error_code_changes_only branch from 58aca9a to 90ca8e1 Compare April 17, 2024 11:32
Copy link
Contributor

@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, will merge it after CI passes. Apply a style format

@emasab emasab merged commit 88b6839 into dev_kip848 Apr 17, 2024
@emasab emasab deleted the dev_kip848_offset_fetch_commit_error_code_changes_only branch April 17, 2024 13:46
emasab pushed a commit that referenced this pull request Apr 18, 2024
…ch (#4681)

- Added new errors to manual commit.
- improvements to OffsetCommit and OffsetFetch error code handling.
anchitj pushed a commit that referenced this pull request Jun 10, 2024
…ch (#4681)

- Added new errors to manual commit.
- improvements to OffsetCommit and OffsetFetch error code handling.
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