From 1b288761fba6be99eb8a179e4247cf05c54a1d92 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Thu, 5 Feb 2026 17:28:41 +0530 Subject: [PATCH 01/20] Mock Broker : Implement share fetch --- src/rdkafka_mock.c | 74 +++++ src/rdkafka_mock.h | 20 ++ src/rdkafka_mock_cgrp.c | 142 ++++++++++ src/rdkafka_mock_handlers.c | 550 ++++++++++++++++++++++++++++++++++++ src/rdkafka_mock_int.h | 92 ++++++ src/rdkafka_op.c | 2 + src/rdkafka_op.h | 13 +- 7 files changed, 892 insertions(+), 1 deletion(-) diff --git a/src/rdkafka_mock.c b/src/rdkafka_mock.c index cdc0445f28..0a7d21a5f0 100644 --- a/src/rdkafka_mock.c +++ b/src/rdkafka_mock.c @@ -2223,6 +2223,47 @@ rd_kafka_mock_partition_set_follower_wmarks(rd_kafka_mock_cluster_t *mcluster, rd_kafka_op_req(mcluster->ops, rko, RD_POLL_INFINITE)); } +rd_kafka_resp_err_t +rd_kafka_mock_partition_produce(rd_kafka_mock_cluster_t *mcluster, + const char *topic, + int32_t partition, + const void *records, + size_t records_size, + const char *transactional_id, + int64_t *base_offset) { + rd_kafka_op_t *rko; + rd_kafka_op_t *reply; + + if (!mcluster || !topic || partition < 0 || !records || + records_size == 0 || records_size > INT32_MAX) { + if (base_offset) + *base_offset = -1; + return RD_KAFKA_RESP_ERR__INVALID_ARG; + } + + rko = rd_kafka_op_new(RD_KAFKA_OP_MOCK); + rko->rko_u.mock.name = rd_strdup(topic); + rko->rko_u.mock.partition = partition; + rko->rko_u.mock.records = + rd_kafkap_bytes_new(records, (int32_t)records_size); + rko->rko_u.mock.base_offset = -1; + if (transactional_id) + rko->rko_u.mock.str = rd_strdup(transactional_id); + rko->rko_u.mock.cmd = RD_KAFKA_MOCK_CMD_PART_PRODUCE; + + reply = rd_kafka_op_req(mcluster->ops, rko, RD_POLL_INFINITE); + if (!reply) { + if (base_offset) + *base_offset = -1; + return RD_KAFKA_RESP_ERR__TIMED_OUT; + } + + if (base_offset) + *base_offset = reply->rko_u.mock.base_offset; + + return rd_kafka_op_err_destroy(reply); +} + rd_kafka_resp_err_t rd_kafka_mock_partition_push_leader_response(rd_kafka_mock_cluster_t *mcluster, const char *topic, @@ -2588,6 +2629,33 @@ rd_kafka_mock_cluster_cmd(rd_kafka_mock_cluster_t *mcluster, mpart->update_follower_end_offset = rd_false; } break; + case RD_KAFKA_MOCK_CMD_PART_PRODUCE: { + rd_kafkap_str_t TransactionalId = RD_KAFKAP_STR_INITIALIZER; + + mpart = rd_kafka_mock_partition_get( + mcluster, rko->rko_u.mock.name, rko->rko_u.mock.partition); + if (!mpart) + return RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; + if (!mpart->leader) + return RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE; + + if (!rko->rko_u.mock.records || + RD_KAFKAP_BYTES_IS_NULL(rko->rko_u.mock.records) || + RD_KAFKAP_BYTES_LEN(rko->rko_u.mock.records) == 0) + return RD_KAFKA_RESP_ERR__INVALID_ARG; + + if (rko->rko_u.mock.str && rko->rko_u.mock.str[0]) { + TransactionalId.str = rko->rko_u.mock.str; + TransactionalId.len = + (int)strlen(rko->rko_u.mock.str); + } + + rko->rko_u.mock.base_offset = -1; + err = rd_kafka_mock_partition_log_append( + mpart, rko->rko_u.mock.records, &TransactionalId, + &rko->rko_u.mock.base_offset); + return err; + } case RD_KAFKA_MOCK_CMD_PART_PUSH_LEADER_RESPONSE: mpart = rd_kafka_mock_partition_get( mcluster, rko->rko_u.mock.name, rko->rko_u.mock.partition); @@ -2708,6 +2776,7 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) { rd_kafka_mock_broker_t *mrkb; rd_kafka_mock_cgrp_classic_t *mcgrp_classic; rd_kafka_mock_cgrp_consumer_t *mcgrp_consumer; + rd_kafka_mock_sgrp_t *msgrp; rd_kafka_mock_coord_t *mcoord; rd_kafka_mock_error_stack_t *errstack; thrd_t dummy_rkb_thread; @@ -2726,6 +2795,9 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) { while ((mcgrp_consumer = TAILQ_FIRST(&mcluster->cgrps_consumer))) rd_kafka_mock_cgrp_consumer_destroy(mcgrp_consumer); + while ((msgrp = TAILQ_FIRST(&mcluster->sgrps_share))) + rd_kafka_mock_sgrp_destroy(msgrp); + while ((mcoord = TAILQ_FIRST(&mcluster->coords))) rd_kafka_mock_coord_destroy(mcluster, mcoord); @@ -2843,6 +2915,8 @@ rd_kafka_mock_cluster_t *rd_kafka_mock_cluster_new(rd_kafka_t *rk, TAILQ_INIT(&mcluster->cgrps_consumer); + TAILQ_INIT(&mcluster->sgrps_share); + TAILQ_INIT(&mcluster->coords); rd_list_init(&mcluster->pids, 16, rd_free); diff --git a/src/rdkafka_mock.h b/src/rdkafka_mock.h index 0b81b312ef..8561206d0c 100644 --- a/src/rdkafka_mock.h +++ b/src/rdkafka_mock.h @@ -292,6 +292,26 @@ rd_kafka_mock_partition_set_follower_wmarks(rd_kafka_mock_cluster_t *mcluster, int64_t lo, int64_t hi); +/** + * @brief Append a record batch to a partition log. + * + * The record batch must use the Kafka record batch format (v2), the same + * encoding as the ProduceRequest Records field. + * + * The topic will be created if it does not exist. + * + * @param transactional_id Optional transactional.id to validate PID/sequence. + * @param base_offset Optional pointer to store the first assigned offset. + */ +RD_EXPORT rd_kafka_resp_err_t +rd_kafka_mock_partition_produce(rd_kafka_mock_cluster_t *mcluster, + const char *topic, + int32_t partition, + const void *records, + size_t records_size, + const char *transactional_id, + int64_t *base_offset); + /** * @brief Push \p cnt Metadata leader response * onto the cluster's stack for the given \p topic and \p partition. diff --git a/src/rdkafka_mock_cgrp.c b/src/rdkafka_mock_cgrp.c index 0c75e003e5..bea8223097 100644 --- a/src/rdkafka_mock_cgrp.c +++ b/src/rdkafka_mock_cgrp.c @@ -1862,6 +1862,148 @@ void rd_kafka_mock_cgrp_consumer_destroy(rd_kafka_mock_cgrp_consumer_t *mcgrp) { rd_free(mcgrp); } +/** + * @brief Destroy share-partition metadata. + */ +static void +rd_kafka_mock_sgrp_partmeta_destroy(rd_kafka_mock_sgrp_partmeta_t *pmeta) { + rd_kafka_mock_sgrp_record_state_t *state; + while ((state = TAILQ_FIRST(&pmeta->inflight))) { + TAILQ_REMOVE(&pmeta->inflight, state, link); + rd_free(state->owner_member_id); + rd_free(state); + } + rd_free(pmeta); +} + +/** + * @brief Destroy share fetch session. + */ +static void rd_kafka_mock_sgrp_fetch_session_destroy( + rd_kafka_mock_sgrp_fetch_session_t *session) { + rd_free(session->member_id); + RD_IF_FREE(session->partitions, rd_kafka_topic_partition_list_destroy); + rd_free(session); +} + +/** + * @brief Check if any share fetch sessions have expired. + */ +static void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts, + void *arg) { + rd_kafka_mock_sgrp_t *sgrp = arg; + rd_kafka_mock_sgrp_fetch_session_t *session, *tmp; + rd_ts_t now = rd_clock(); + + (void)rkts; + + TAILQ_FOREACH_SAFE(session, &sgrp->fetch_sessions, link, tmp) { + if (session->ts_last_activity + + (sgrp->session_timeout_ms * 1000) > + now) + continue; + + TAILQ_REMOVE(&sgrp->fetch_sessions, session, link); + sgrp->fetch_session_cnt--; + rd_kafka_mock_sgrp_fetch_session_destroy(session); + } +} + +/** + * @brief Find share group in cluster by GroupId. + * + * @locks mcluster->lock MUST be held. + */ +rd_kafka_mock_sgrp_t * +rd_kafka_mock_sgrp_find(rd_kafka_mock_cluster_t *mcluster, + const rd_kafkap_str_t *GroupId) { + rd_kafka_mock_sgrp_t *sgrp; + TAILQ_FOREACH(sgrp, &mcluster->sgrps_share, link) { + if (!rd_kafkap_str_cmp_str(GroupId, sgrp->id)) + return sgrp; + } + + return NULL; +} + +/** + * @brief Find or create a share group for ShareFetch. + * + * @locks mcluster->lock MUST be held. + */ +rd_kafka_mock_sgrp_t * +rd_kafka_mock_sgrp_get(rd_kafka_mock_cluster_t *mcluster, + const rd_kafkap_str_t *GroupId) { + rd_kafka_mock_sgrp_t *sgrp; + + sgrp = rd_kafka_mock_sgrp_find(mcluster, GroupId); + if (sgrp) + return sgrp; + + sgrp = rd_calloc(1, sizeof(*sgrp)); + sgrp->cluster = mcluster; + sgrp->id = RD_KAFKAP_STR_DUP(GroupId); + sgrp->group_epoch = 1; + sgrp->session_timeout_ms = + mcluster->defaults.group_consumer_session_timeout_ms; + TAILQ_INIT(&sgrp->members); + TAILQ_INIT(&sgrp->partitions); + TAILQ_INIT(&sgrp->fetch_sessions); + rd_kafka_timer_start(&mcluster->timers, &sgrp->session_tmr, + 1000 * 1000 /*1s*/, + rd_kafka_mock_sgrp_fetch_session_tmr_cb, sgrp); + + TAILQ_INSERT_TAIL(&mcluster->sgrps_share, sgrp, link); + + return sgrp; +} + +/** + * @brief Destroy share group and all of its ShareFetch state. + * + * @locks mcluster->lock MUST be held. + */ +void rd_kafka_mock_sgrp_destroy(rd_kafka_mock_sgrp_t *sgrp) { + rd_kafka_mock_sgrp_member_t *member; + rd_kafka_mock_sgrp_partmeta_t *pmeta; + rd_kafka_mock_sgrp_fetch_session_t *session; + + TAILQ_REMOVE(&sgrp->cluster->sgrps_share, sgrp, link); + + rd_kafka_timer_stop(&sgrp->cluster->timers, &sgrp->session_tmr, rd_true); + + while ((member = TAILQ_FIRST(&sgrp->members))) { + TAILQ_REMOVE(&sgrp->members, member, link); + sgrp->member_cnt--; + rd_free(member->id); + RD_IF_FREE(member->instance_id, rd_free); + RD_IF_FREE(member->target_assignment, + rd_kafka_topic_partition_list_destroy); + RD_IF_FREE(member->current_assignment, + rd_kafka_topic_partition_list_destroy); + RD_IF_FREE(member->returned_assignment, + rd_kafka_topic_partition_list_destroy); + RD_IF_FREE(member->subscribed_topics, rd_list_destroy_free); + RD_IF_FREE(member->subscribed_topic_names, rd_list_destroy_free); + RD_IF_FREE(member->subscribed_topic_regex, rd_free); + rd_free(member); + } + + while ((pmeta = TAILQ_FIRST(&sgrp->partitions))) { + TAILQ_REMOVE(&sgrp->partitions, pmeta, link); + rd_kafka_mock_sgrp_partmeta_destroy(pmeta); + } + + while ((session = TAILQ_FIRST(&sgrp->fetch_sessions))) { + TAILQ_REMOVE(&sgrp->fetch_sessions, session, link); + sgrp->fetch_session_cnt--; + rd_kafka_mock_sgrp_fetch_session_destroy(session); + } + + rd_free(sgrp->id); + rd_free(sgrp); +} + /** * @brief A client connection closed, check if any cgrp has any state * for this connection that needs to be cleared. diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index ad509ecceb..8899c18cd3 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -2993,6 +2993,555 @@ rd_kafka_mock_handle_ConsumerGroupHeartbeat(rd_kafka_mock_connection_t *mconn, return -1; } +/** + * @brief Handle ShareFetch (KIP-932) - request parsing only. + */ +static rd_bool_t +rd_kafka_mock_tplist_equal_by_id(rd_kafka_topic_partition_list_t *a, + rd_kafka_topic_partition_list_t *b) { + int i; + + if (!a && !b) + return rd_true; + if (!a || !b) + return rd_false; + if (a->cnt != b->cnt) + return rd_false; + + rd_kafka_topic_partition_list_sort_by_topic_id(a); + rd_kafka_topic_partition_list_sort_by_topic_id(b); + + for (i = 0; i < a->cnt; i++) { + const rd_kafka_topic_partition_t *pa = &a->elems[i]; + const rd_kafka_topic_partition_t *pb = &b->elems[i]; + rd_kafka_Uuid_t ta = + rd_kafka_topic_partition_get_topic_id(pa); + rd_kafka_Uuid_t tb = + rd_kafka_topic_partition_get_topic_id(pb); + + if (pa->partition != pb->partition) + return rd_false; + if (rd_kafka_Uuid_cmp(ta, tb) != 0) + return rd_false; + } + + return rd_true; +} + +static rd_kafka_mock_sgrp_partmeta_t * +rd_kafka_mock_sgrp_partmeta_find(rd_kafka_mock_sgrp_t *sgrp, + rd_kafka_Uuid_t topic_id, + int32_t partition) { + rd_kafka_mock_sgrp_partmeta_t *pmeta; + + TAILQ_FOREACH(pmeta, &sgrp->partitions, link) { + if (pmeta->partition != partition) + continue; + if (!rd_kafka_Uuid_cmp(pmeta->topic_id, topic_id)) + return pmeta; + } + + return NULL; +} + +static rd_kafka_mock_sgrp_partmeta_t * +rd_kafka_mock_sgrp_partmeta_get(rd_kafka_mock_sgrp_t *sgrp, + rd_kafka_Uuid_t topic_id, + int32_t partition, + const rd_kafka_mock_partition_t *mpart) { + rd_kafka_mock_sgrp_partmeta_t *pmeta; + int64_t log_start; + int64_t log_end; + + pmeta = rd_kafka_mock_sgrp_partmeta_find(sgrp, topic_id, partition); + if (pmeta) { + log_start = mpart->start_offset; + log_end = mpart->end_offset; + if (log_start > pmeta->spso) + pmeta->spso = log_start; + if (log_end > log_start) { + int64_t new_speo = log_end - 1; + if (new_speo > pmeta->speo) + pmeta->speo = new_speo; + } + return pmeta; + } + + pmeta = rd_calloc(1, sizeof(*pmeta)); + pmeta->topic_id = topic_id; + pmeta->partition = partition; + pmeta->spso = mpart->start_offset; + if (mpart->end_offset > mpart->start_offset) + pmeta->speo = mpart->end_offset - 1; + else + pmeta->speo = mpart->start_offset - 1; + TAILQ_INIT(&pmeta->inflight); + + TAILQ_INSERT_TAIL(&sgrp->partitions, pmeta, link); + sgrp->partition_cnt++; + + return pmeta; +} + +static rd_kafka_mock_sgrp_record_state_t * +rd_kafka_mock_sgrp_record_state_find(rd_kafka_mock_sgrp_partmeta_t *pmeta, + int64_t offset) { + rd_kafka_mock_sgrp_record_state_t *state; + + TAILQ_FOREACH(state, &pmeta->inflight, link) { + if (state->offset == offset) + return state; + } + + return NULL; +} + +static rd_kafka_mock_sgrp_record_state_t * +rd_kafka_mock_sgrp_record_state_get(rd_kafka_mock_sgrp_partmeta_t *pmeta, + int64_t offset) { + rd_kafka_mock_sgrp_record_state_t *state; + + state = rd_kafka_mock_sgrp_record_state_find(pmeta, offset); + if (state) + return state; + + state = rd_calloc(1, sizeof(*state)); + state->offset = offset; + state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE; + TAILQ_INSERT_TAIL(&pmeta->inflight, state, link); + pmeta->inflight_cnt++; + + return state; +} + +static int32_t +rd_kafka_mock_msgset_est_record_size(const rd_kafka_mock_msgset_t *mset) { + int64_t record_cnt = mset->last_offset - mset->first_offset + 1; + int32_t size; + + if (record_cnt <= 0) + return 1; + + size = (int32_t)(RD_KAFKAP_BYTES_LEN(&mset->bytes) / record_cnt); + if (size <= 0) + size = 1; + + return size; +} + +static void +rd_kafka_mock_sgrp_acquire_available_offsets( + rd_kafka_mock_sgrp_partmeta_t *pmeta, + const rd_kafka_mock_partition_t *mpart, + const rd_kafkap_str_t *member_id, + rd_ts_t lock_expiry_ts, + int64_t *remaining_records, + int64_t *remaining_bytes, + int *acquired_cnt, + int64_t *acquired_bytes) { + int64_t offset; + + for (offset = pmeta->spso; offset <= pmeta->speo; offset++) { + const rd_kafka_mock_msgset_t *mset; + rd_kafka_mock_sgrp_record_state_t *state; + int32_t est_size; + + if (remaining_records && *remaining_records == 0) + break; + if (remaining_bytes && *remaining_bytes == 0) + break; + + state = rd_kafka_mock_sgrp_record_state_find(pmeta, offset); + if (state && + state->state != RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE) + continue; + + mset = rd_kafka_mock_msgset_find(mpart, offset, rd_false); + if (!mset) + continue; + + est_size = rd_kafka_mock_msgset_est_record_size(mset); + if (remaining_bytes && *remaining_bytes > 0 && + est_size > *remaining_bytes) + break; + + state = rd_kafka_mock_sgrp_record_state_get(pmeta, offset); + state->state = RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED; + state->lock_expiry_ts = lock_expiry_ts; + state->delivery_count++; + RD_IF_FREE(state->owner_member_id, rd_free); + state->owner_member_id = RD_KAFKAP_STR_DUP(member_id); + + (*acquired_cnt)++; + *acquired_bytes += est_size; + if (remaining_records && *remaining_records > 0) + (*remaining_records)--; + if (remaining_bytes && *remaining_bytes > 0) + (*remaining_bytes) -= est_size; + } +} + +static void +rd_kafka_mock_sgrp_partmeta_prune_archived(rd_kafka_mock_sgrp_partmeta_t *pmeta) { + rd_kafka_mock_sgrp_record_state_t *state, *tmp; + + TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { + if (state->state != RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED) + continue; + if (state->offset <= pmeta->speo) + continue; + + TAILQ_REMOVE(&pmeta->inflight, state, link); + pmeta->inflight_cnt--; + rd_free(state->owner_member_id); + rd_free(state); + } +} + +static const rd_kafka_mock_msgset_t * +rd_kafka_mock_sgrp_first_acquired_msgset( + rd_kafka_mock_sgrp_partmeta_t *pmeta, + const rd_kafka_mock_partition_t *mpart, + const rd_kafkap_str_t *member_id, + rd_ts_t now) { + int64_t offset; + + for (offset = pmeta->spso; offset <= pmeta->speo; offset++) { + rd_kafka_mock_sgrp_record_state_t *state = + rd_kafka_mock_sgrp_record_state_find(pmeta, offset); + const rd_kafka_mock_msgset_t *mset; + + if (!state || + state->state != RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) + continue; + + if (state->lock_expiry_ts && + state->lock_expiry_ts <= now) + continue; + + if (rd_kafkap_str_cmp_str(member_id, state->owner_member_id)) + continue; + + mset = rd_kafka_mock_msgset_find(mpart, offset, rd_false); + if (mset) + return mset; + } + + return NULL; +} + +static int +rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, + rd_kafka_buf_t *rkbuf) { + const rd_bool_t log_decode_errors = rd_true; + rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster; + rd_kafka_buf_t *resp = rd_kafka_mock_buf_new_response(rkbuf); + rd_kafkap_str_t GroupId, MemberId; + int32_t SessionId = -1, SessionEpoch = -1, MaxWaitMs = 0, MinBytes = 0, + MaxBytes = 0, MaxRecords = 0; + int32_t TopicsCnt; + rd_kafka_topic_partition_list_t *requested_partitions = NULL; + rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; + rd_kafka_mock_sgrp_t *sgrp = NULL; + rd_kafka_mock_sgrp_fetch_session_t *session = NULL; + static int32_t next_session_id = 1; + + (void)log_decode_errors; + + rd_kafka_buf_read_str(rkbuf, &GroupId); + rd_kafka_buf_read_str(rkbuf, &MemberId); + rd_kafka_buf_read_i32(rkbuf, &SessionId); + rd_kafka_buf_read_i32(rkbuf, &SessionEpoch); + rd_kafka_buf_read_i32(rkbuf, &MaxWaitMs); + rd_kafka_buf_read_i32(rkbuf, &MinBytes); + rd_kafka_buf_read_i32(rkbuf, &MaxBytes); + rd_kafka_buf_read_i32(rkbuf, &MaxRecords); + + requested_partitions = rd_kafka_topic_partition_list_new(0); + + rd_kafka_buf_read_arraycnt(rkbuf, &TopicsCnt, RD_KAFKAP_TOPICS_MAX); + while (TopicsCnt-- > 0) { + rd_kafka_Uuid_t TopicId = RD_KAFKA_UUID_ZERO; + int32_t PartitionCnt; + + rd_kafka_buf_read_uuid(rkbuf, &TopicId); + rd_kafka_buf_read_arraycnt(rkbuf, &PartitionCnt, + RD_KAFKAP_PARTITIONS_MAX); + + while (PartitionCnt-- > 0) { + int32_t Partition; + int32_t PartitionMaxBytes; + int32_t PartitionMaxRecords; + rd_kafka_topic_partition_t *rktpar; + + rd_kafka_buf_read_i32(rkbuf, &Partition); + rd_kafka_buf_read_i32(rkbuf, &PartitionMaxBytes); + rd_kafka_buf_read_i32(rkbuf, &PartitionMaxRecords); + + rktpar = rd_kafka_topic_partition_list_add( + requested_partitions, "", Partition); + rd_kafka_topic_partition_set_topic_id(rktpar, TopicId); + + rd_kafka_buf_skip_tags(rkbuf); + } + + rd_kafka_buf_skip_tags(rkbuf); + } + + rd_kafka_buf_skip_tags(rkbuf); + + rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK", + "ShareFetch parsed: group %.*s member %.*s " + "session_id %" PRId32 " session_epoch %" PRId32 + " max_wait %" PRId32 " min_bytes %" PRId32 + " max_bytes %" PRId32 " max_records %" PRId32, + RD_KAFKAP_STR_PR(&GroupId), RD_KAFKAP_STR_PR(&MemberId), + SessionId, SessionEpoch, MaxWaitMs, MinBytes, MaxBytes, + MaxRecords); + + err = rd_kafka_mock_next_request_error(mconn, resp); + + if (!err) { + int64_t remaining_records = + MaxRecords > 0 ? (int64_t)MaxRecords : -1; + int64_t remaining_bytes = + MaxBytes > 0 ? (int64_t)MaxBytes : -1; + int acquired_cnt = 0; + int64_t acquired_bytes = 0; + rd_ts_t now = rd_clock(); + + // Handle ShareFetch session management + mtx_lock(&mcluster->lock); + sgrp = rd_kafka_mock_sgrp_get(mcluster, &GroupId); + + if (SessionId < 0) { + session = + rd_calloc(1, sizeof(*session)); + session->member_id = RD_KAFKAP_STR_DUP(&MemberId); + session->session_id = next_session_id++; + session->session_epoch = 0; + session->ts_last_activity = rd_clock(); + session->partitions = + rd_kafka_topic_partition_list_copy( + requested_partitions); + TAILQ_INSERT_TAIL(&sgrp->fetch_sessions, session, link); + sgrp->fetch_session_cnt++; + } else { + TAILQ_FOREACH(session, &sgrp->fetch_sessions, link) { + if (session->session_id != SessionId) + continue; + if (rd_kafkap_str_cmp_str(&MemberId, + session->member_id)) + continue; + break; + } + + if (!session) { + err = + RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH; + } else if (SessionEpoch != -1 && + SessionEpoch != session->session_epoch) { + err = + RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH; + } else { + if (!rd_kafka_mock_tplist_equal_by_id( + requested_partitions, + session->partitions)) { + session->session_epoch++; + RD_IF_FREE( + session->partitions, + rd_kafka_topic_partition_list_destroy); + session->partitions = + rd_kafka_topic_partition_list_copy( + requested_partitions); + } + session->ts_last_activity = rd_clock(); + } + } + + if (!err && sgrp) { + rd_kafka_topic_partition_t *rktpar; + RD_KAFKA_TPLIST_FOREACH(rktpar, requested_partitions) { + rd_kafka_Uuid_t topic_id = + rd_kafka_topic_partition_get_topic_id( + rktpar); + rd_kafka_mock_topic_t *mtopic = + rd_kafka_mock_topic_find_by_id(mcluster, + topic_id); + rd_kafka_mock_partition_t *mpart; + + if (!mtopic) { + err = + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; + break; + } + + mpart = rd_kafka_mock_partition_find( + mtopic, rktpar->partition); + if (!mpart) { + err = + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; + break; + } + + rd_kafka_mock_sgrp_partmeta_t *pmeta = + rd_kafka_mock_sgrp_partmeta_get( + sgrp, topic_id, rktpar->partition, + mpart); + rd_kafka_mock_sgrp_partmeta_prune_archived( + pmeta); + rd_kafka_mock_sgrp_acquire_available_offsets( + pmeta, mpart, + &MemberId, + now + + (sgrp->session_timeout_ms * 1000), + MaxRecords > 0 ? &remaining_records : NULL, + MaxBytes > 0 ? &remaining_bytes : NULL, + &acquired_cnt, &acquired_bytes); + } + } + + rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK", + "ShareFetch acquired: %d records, %" PRId64 + " bytes", + acquired_cnt, acquired_bytes); + + /* Response: ThrottleTimeMs */ + rd_kafka_buf_write_i32(resp, 0); + /* Response: ErrorCode */ + rd_kafka_buf_write_i16(resp, err); + /* Response: SessionId */ + rd_kafka_buf_write_i32( + resp, session ? session->session_id : -1); + /* Response: SessionEpoch */ + rd_kafka_buf_write_i32( + resp, session ? session->session_epoch : -1); + + rd_kafka_topic_partition_list_sort_by_topic_id( + requested_partitions); + + { + int i = 0; + int topic_cnt = 0; + rd_kafka_Uuid_t current_topic = RD_KAFKA_UUID_ZERO; + + for (i = 0; i < requested_partitions->cnt; i++) { + rd_kafka_topic_partition_t *rktpar = + &requested_partitions->elems[i]; + rd_kafka_Uuid_t topic_id = + rd_kafka_topic_partition_get_topic_id( + rktpar); + if (i == 0 || + rd_kafka_Uuid_cmp(topic_id, + current_topic) != 0) { + topic_cnt++; + current_topic = topic_id; + } + } + + /* Response: #Topics */ + rd_kafka_buf_write_arraycnt(resp, topic_cnt); + + i = 0; + while (i < requested_partitions->cnt) { + int j; + rd_kafka_Uuid_t topic_id = + rd_kafka_topic_partition_get_topic_id( + &requested_partitions->elems[i]); + int part_cnt = 0; + + for (j = i; j < requested_partitions->cnt; + j++) { + rd_kafka_Uuid_t next_topic_id = + rd_kafka_topic_partition_get_topic_id( + &requested_partitions->elems[j]); + if (rd_kafka_Uuid_cmp(topic_id, + next_topic_id) != + 0) + break; + part_cnt++; + } + + /* Response: TopicId */ + rd_kafka_buf_write_uuid(resp, &topic_id); + /* Response: #Partitions */ + rd_kafka_buf_write_arraycnt(resp, part_cnt); + + for (j = i; j < i + part_cnt; j++) { + rd_kafka_topic_partition_t *rktpar = + &requested_partitions->elems[j]; + rd_kafka_mock_topic_t *mtopic = + rd_kafka_mock_topic_find_by_id( + mcluster, topic_id); + rd_kafka_mock_partition_t *mpart = + mtopic ? rd_kafka_mock_partition_find( + mtopic, + rktpar->partition) + : NULL; + rd_kafka_mock_sgrp_partmeta_t *pmeta = + mpart + ? rd_kafka_mock_sgrp_partmeta_find( + sgrp, topic_id, + rktpar->partition) + : NULL; + const rd_kafka_mock_msgset_t *mset = + (mpart && pmeta) + ? rd_kafka_mock_sgrp_first_acquired_msgset( + pmeta, mpart, &MemberId, + now) + : NULL; + rd_kafka_resp_err_t part_err = + mpart ? RD_KAFKA_RESP_ERR_NO_ERROR + : RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; + + /* Response: Partition */ + rd_kafka_buf_write_i32( + resp, rktpar->partition); + /* Response: ErrorCode */ + rd_kafka_buf_write_i16(resp, part_err); + /* Response: Records */ + rd_kafka_buf_write_kbytes( + resp, mset ? &mset->bytes : NULL); + /* Response: Partition tags */ + rd_kafka_buf_write_tags_empty(resp); + } + + /* Response: Topic tags */ + rd_kafka_buf_write_tags_empty(resp); + + i += part_cnt; + } + } + + /* Response: Top-level tags */ + rd_kafka_buf_write_tags_empty(resp); + + mtx_unlock(&mcluster->lock); + + rd_kafka_mock_connection_send_response(mconn, resp); + + rd_kafka_topic_partition_list_destroy(requested_partitions); + return 0; + } + + /* Error response */ + rd_kafka_buf_write_i32(resp, 0); + rd_kafka_buf_write_i16(resp, err); + rd_kafka_buf_write_i32(resp, -1); + rd_kafka_buf_write_i32(resp, -1); + rd_kafka_buf_write_arraycnt(resp, 0); + rd_kafka_buf_write_tags_empty(resp); + + rd_kafka_mock_connection_send_response(mconn, resp); + rd_kafka_topic_partition_list_destroy(requested_partitions); + return 0; + +err_parse: + RD_IF_FREE(requested_partitions, rd_kafka_topic_partition_list_destroy); + rd_kafka_buf_destroy(resp); + return -1; +} + /** * @brief Default request handlers */ @@ -3029,6 +3578,7 @@ const struct rd_kafka_mock_api_handler {0, 0, 0, rd_kafka_mock_handle_GetTelemetrySubscriptions}, [RD_KAFKAP_PushTelemetry] = {0, 0, 0, rd_kafka_mock_handle_PushTelemetry}, + [RD_KAFKAP_ShareFetch] = {0, 0, 0, rd_kafka_mock_handle_ShareFetch}, }; diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index 2ef7a2a339..78239a4aa7 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -179,6 +179,88 @@ typedef struct rd_kafka_mock_cgrp_consumer_member_s { rd_kafka_mock_cgrp_consumer_t *mcgrp; /**< Consumer group */ } rd_kafka_mock_cgrp_consumer_member_t; +/** + * @struct Share group member (KIP-932). + */ +typedef struct rd_kafka_mock_sgrp_member_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_member_s) link; + char *id; /**< MemberId */ + char *instance_id; /**< Group instance id */ + rd_ts_t ts_last_activity; /**< Last activity */ + int32_t current_member_epoch; /**< Current member epoch */ + int32_t target_member_epoch; /**< Target member epoch */ + rd_kafka_topic_partition_list_t *current_assignment; + rd_kafka_topic_partition_list_t *target_assignment; + rd_kafka_topic_partition_list_t *returned_assignment; + rd_list_t *subscribed_topics; + rd_list_t *subscribed_topic_names; + char *subscribed_topic_regex; + rd_bool_t left_static_membership; + struct rd_kafka_mock_connection_s *conn; + struct rd_kafka_mock_sgrp_s *sgrp; +} rd_kafka_mock_sgrp_member_t; + +/** + * @brief Share record state. + */ +enum rd_kafka_mock_sgrp_record_state_e { + RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE = 0, + RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED = 1, + RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED = 2 +}; + +typedef struct rd_kafka_mock_sgrp_record_state_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_record_state_s) link; + int64_t offset; + char *owner_member_id; + rd_ts_t lock_expiry_ts; + int32_t delivery_count; + enum rd_kafka_mock_sgrp_record_state_e state; +} rd_kafka_mock_sgrp_record_state_t; + +/** + * @brief Share partition metadata. + */ +typedef struct rd_kafka_mock_sgrp_partmeta_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_partmeta_s) link; + rd_kafka_Uuid_t topic_id; + int32_t partition; + int64_t spso; + int64_t speo; + TAILQ_HEAD(, rd_kafka_mock_sgrp_record_state_s) inflight; + int inflight_cnt; +} rd_kafka_mock_sgrp_partmeta_t; + +/** + * @brief Share fetch session. + */ +typedef struct rd_kafka_mock_sgrp_fetch_session_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_fetch_session_s) link; + char *member_id; + int32_t session_id; + int32_t session_epoch; + rd_ts_t ts_last_activity; + rd_kafka_topic_partition_list_t *partitions; +} rd_kafka_mock_sgrp_fetch_session_t; + +/** + * @brief Share group (KIP-932). + */ +typedef struct rd_kafka_mock_sgrp_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_s) link; + struct rd_kafka_mock_cluster_s *cluster; + char *id; + int32_t group_epoch; + int session_timeout_ms; + rd_kafka_timer_t session_tmr; + TAILQ_HEAD(, rd_kafka_mock_sgrp_member_s) members; + int member_cnt; + TAILQ_HEAD(, rd_kafka_mock_sgrp_partmeta_s) partitions; + int partition_cnt; + TAILQ_HEAD(, rd_kafka_mock_sgrp_fetch_session_s) fetch_sessions; + int fetch_session_cnt; +} rd_kafka_mock_sgrp_t; + /** * @struct TransactionalId + PID (+ optional sequence state) @@ -429,6 +511,8 @@ struct rd_kafka_mock_cluster_s { TAILQ_HEAD(, rd_kafka_mock_cgrp_consumer_s) cgrps_consumer; + TAILQ_HEAD(, rd_kafka_mock_sgrp_s) sgrps_share; + /** Explicit coordinators (set with mock_set_coordinator()) */ TAILQ_HEAD(, rd_kafka_mock_coord_s) coords; @@ -541,6 +625,14 @@ rd_kafka_mock_topic_t * rd_kafka_mock_topic_find_by_id(const rd_kafka_mock_cluster_t *mcluster, rd_kafka_Uuid_t id); +rd_kafka_mock_sgrp_t * +rd_kafka_mock_sgrp_find(rd_kafka_mock_cluster_t *mcluster, + const rd_kafkap_str_t *GroupId); +rd_kafka_mock_sgrp_t * +rd_kafka_mock_sgrp_get(rd_kafka_mock_cluster_t *mcluster, + const rd_kafkap_str_t *GroupId); +void rd_kafka_mock_sgrp_destroy(rd_kafka_mock_sgrp_t *sgrp); + rd_kafka_mock_broker_t * rd_kafka_mock_cluster_get_coord(rd_kafka_mock_cluster_t *mcluster, rd_kafka_coordtype_t KeyType, diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index 0cce441768..e15593cf1e 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -481,6 +481,8 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) { case RD_KAFKA_OP_MOCK: RD_IF_FREE(rko->rko_u.mock.name, rd_free); RD_IF_FREE(rko->rko_u.mock.str, rd_free); + if (rko->rko_u.mock.records) + rd_kafkap_bytes_destroy(rko->rko_u.mock.records); if (rko->rko_u.mock.metrics) { int64_t i; for (i = 0; i < rko->rko_u.mock.hi; i++) diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index ef0e4c4d73..2a78018183 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -591,6 +591,7 @@ struct rd_kafka_op_s { RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER, RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER_WMARKS, RD_KAFKA_MOCK_CMD_PART_PUSH_LEADER_RESPONSE, + RD_KAFKA_MOCK_CMD_PART_PRODUCE, RD_KAFKA_MOCK_CMD_BROKER_SET_UPDOWN, RD_KAFKA_MOCK_CMD_BROKER_SET_RTT, RD_KAFKA_MOCK_CMD_BROKER_SET_RACK, @@ -612,15 +613,19 @@ struct rd_kafka_op_s { * BROKER_SET_RACK * COORD_SET (key_type) * PART_PUSH_LEADER_RESPONSE + * PART_PRODUCE */ char *str; /**< For: - * COORD_SET (key) */ + * COORD_SET (key) + * PART_PRODUCE + */ int32_t partition; /**< For: * PART_SET_FOLLOWER * PART_SET_FOLLOWER_WMARKS * PART_SET_LEADER * APIVERSION_SET (ApiKey) * PART_PUSH_LEADER_RESPONSE + * PART_PRODUCE */ int32_t broker_id; /**< For: * PART_SET_FOLLOWER @@ -652,6 +657,12 @@ struct rd_kafka_op_s { */ char **metrics; /**< Metrics requested, for: * REQUESTED_METRICS_SET */ + rd_kafkap_bytes_t *records; /**< Records, for: + * PART_PRODUCE + */ + int64_t base_offset; /**< BaseOffset for: + * PART_PRODUCE reply + */ } mock; struct { From d5521e4ee7a4fc7b460baa0f974e31414994093a Mon Sep 17 00:00:00 2001 From: Ankith-Confluent Date: Mon, 2 Feb 2026 17:33:38 +0530 Subject: [PATCH 02/20] ShareGroupHeartBeat MVP --- src/CMakeLists.txt | 1 + src/Makefile | 2 +- src/rdkafka_mock.c | 6 + src/rdkafka_mock_handlers.c | 204 +++++++++++++++++ src/rdkafka_mock_int.h | 75 ++++++ src/rdkafka_mock_sharegrp.c | 440 ++++++++++++++++++++++++++++++++++++ tests/CMakeLists.txt | 1 + win32/librdkafka.vcxproj | 1 + 8 files changed, 729 insertions(+), 1 deletion(-) create mode 100644 src/rdkafka_mock_sharegrp.c diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index bbe63cff48..58a8236516 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -51,6 +51,7 @@ set( rdkafka_mock.c rdkafka_mock_handlers.c rdkafka_mock_cgrp.c + rdkafka_mock_sharegrp.c rdkafka_error.c rdkafka_fetcher.c rdkafka_telemetry.c diff --git a/src/Makefile b/src/Makefile index 0d0635ce30..87de298cbe 100644 --- a/src/Makefile +++ b/src/Makefile @@ -57,7 +57,7 @@ SRCS= rdkafka.c rdkafka_broker.c rdkafka_msg.c rdkafka_topic.c \ rdkafka_background.c rdkafka_idempotence.c rdkafka_cert.c \ rdkafka_txnmgr.c rdkafka_coord.c rdbase64.c \ rdvarint.c rdbuf.c rdmap.c rdunittest.c \ - rdkafka_mock.c rdkafka_mock_handlers.c rdkafka_mock_cgrp.c \ + rdkafka_mock.c rdkafka_mock_handlers.c rdkafka_mock_cgrp.c rdkafka_mock_sharegrp.c \ rdkafka_error.c rdkafka_fetcher.c rdkafka_telemetry.c \ rdkafka_telemetry_encode.c rdkafka_telemetry_decode.c \ nanopb/pb_encode.c nanopb/pb_decode.c nanopb/pb_common.c \ diff --git a/src/rdkafka_mock.c b/src/rdkafka_mock.c index 0a7d21a5f0..3c8b00988f 100644 --- a/src/rdkafka_mock.c +++ b/src/rdkafka_mock.c @@ -2777,6 +2777,7 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) { rd_kafka_mock_cgrp_classic_t *mcgrp_classic; rd_kafka_mock_cgrp_consumer_t *mcgrp_consumer; rd_kafka_mock_sgrp_t *msgrp; + rd_kafka_mock_sharegroup_t *mshgrp; rd_kafka_mock_coord_t *mcoord; rd_kafka_mock_error_stack_t *errstack; thrd_t dummy_rkb_thread; @@ -2798,6 +2799,9 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) { while ((msgrp = TAILQ_FIRST(&mcluster->sgrps_share))) rd_kafka_mock_sgrp_destroy(msgrp); + while ((mshgrp = TAILQ_FIRST(&mcluster->sharegrps))) + rd_kafka_mock_sharegroup_destroy(mshgrp); + while ((mcoord = TAILQ_FIRST(&mcluster->coords))) rd_kafka_mock_coord_destroy(mcluster, mcoord); @@ -2917,6 +2921,8 @@ rd_kafka_mock_cluster_t *rd_kafka_mock_cluster_new(rd_kafka_t *rk, TAILQ_INIT(&mcluster->sgrps_share); + rd_kafka_mock_sharegrps_init(mcluster); + TAILQ_INIT(&mcluster->coords); rd_list_init(&mcluster->pids, 16, rd_free); diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 8899c18cd3..1fe1e76ea9 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -3542,6 +3542,208 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, return -1; } +/** + * @brief Helper to write assignment TopicPartitions to ShareGroupHeartbeat + * response. + */ +static void rd_kafka_mock_handle_ShareGroupHeartbeat_write_TopicPartitions( + rd_kafka_buf_t *resp, + rd_kafka_topic_partition_list_t *assignment) { + const rd_kafka_topic_partition_field_t fields[] = { + RD_KAFKA_TOPIC_PARTITION_FIELD_PARTITION, + RD_KAFKA_TOPIC_PARTITION_FIELD_END}; + + rd_kafka_topic_partition_list_sort_by_topic_id(assignment); + rd_kafka_buf_write_topic_partitions( + resp, assignment, rd_false /* don't skip invalid offsets */, + rd_false /* any offset */, rd_true /* use topic id */, + rd_false /* don't use topic name */, fields); +} + +/** + * @brief Handle ShareGroupHeartbeat request (API Key 76). + */ +static int +rd_kafka_mock_handle_ShareGroupHeartbeat(rd_kafka_mock_connection_t *mconn, + rd_kafka_buf_t *rkbuf) { + const rd_bool_t log_decode_errors = rd_true; + rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster; + rd_kafka_buf_t *resp; + rd_kafkap_str_t GroupId, MemberId, RackId; + rd_kafkap_str_t *SubscribedTopicNames = NULL; + int32_t MemberEpoch, SubscribedTopicNamesCnt; + int32_t i; + rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; + rd_kafka_mock_sharegroup_t *mshgrp = NULL; + rd_kafka_mock_sharegroup_member_t *member = NULL; + rd_bool_t assignment_changed = rd_false; + + resp = rd_kafka_mock_buf_new_response(rkbuf); + + /* GroupId */ + rd_kafka_buf_read_str(rkbuf, &GroupId); + + /* MemberId */ + rd_kafka_buf_read_str(rkbuf, &MemberId); + + /* MemberEpoch */ + rd_kafka_buf_read_i32(rkbuf, &MemberEpoch); + + /* RackId (nullable) */ + rd_kafka_buf_read_str(rkbuf, &RackId); + + /* SubscribedTopicNames array (nullable) */ + rd_kafka_buf_read_arraycnt(rkbuf, &SubscribedTopicNamesCnt, + RD_KAFKAP_TOPICS_MAX); + if (SubscribedTopicNamesCnt >= 0) { + SubscribedTopicNames = rd_calloc( + SubscribedTopicNamesCnt > 0 ? SubscribedTopicNamesCnt : 1, + sizeof(rd_kafkap_str_t)); + for (i = 0; i < SubscribedTopicNamesCnt; i++) { + rd_kafka_buf_read_str(rkbuf, &SubscribedTopicNames[i]); + } + } + + { + mtx_lock(&mcluster->lock); + + mshgrp = rd_kafka_mock_sharegroup_get(mcluster, &GroupId); + + if (MemberEpoch == -1) { + /* LEAVE: Member wants to leave */ + member = rd_kafka_mock_sharegroup_member_find( + mshgrp, &MemberId); + if (member) { + rd_kafka_mock_sharegroup_member_destroy(mshgrp, + member); + member = NULL; + assignment_changed = rd_true; + } + + } else if (MemberEpoch == 0) { + /* JOIN: New member wants to join */ + member = rd_kafka_mock_sharegroup_member_get( + mshgrp, &MemberId, MemberEpoch, mconn); + + if (member) { + if (rd_kafka_mock_sharegroup_member_subscribed_topic_names_set( + member, SubscribedTopicNames, + SubscribedTopicNamesCnt)) { + assignment_changed = rd_true; + } else { + /* New member always triggers + * recalculation */ + assignment_changed = rd_true; + } + MemberEpoch = member->member_epoch; + } else { + err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID; + } + + } else { + /* HEARTBEAT: Existing member heartbeat */ + member = rd_kafka_mock_sharegroup_member_find( + mshgrp, &MemberId); + if (!member) { + err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID; + } else if (MemberEpoch > member->member_epoch) { + /* Client epoch is ahead of server - indicates + * a bug or stale coordinator. */ + err = RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH; + } else if (MemberEpoch < member->member_epoch) { + /* Client epoch is behind. Allow if it matches + * the previous epoch (response with bumped + * epoch may have been lost). Otherwise fence. + */ + if (MemberEpoch != + member->previous_member_epoch) { + err = + RD_KAFKA_RESP_ERR_FENCED_MEMBER_EPOCH; + } else { + /* Accept previous epoch - client is + * catching up */ + member->conn = mconn; + MemberEpoch = member->member_epoch; + } + } else { + /* Epoch matches - normal heartbeat */ + /* Check for subscription changes */ + if (SubscribedTopicNamesCnt >= 0 && + rd_kafka_mock_sharegroup_member_subscribed_topic_names_set( + member, SubscribedTopicNames, + SubscribedTopicNamesCnt)) { + assignment_changed = rd_true; + } + member->conn = mconn; + MemberEpoch = member->member_epoch; + } + } + + /* Recalculate assignments if needed */ + if (assignment_changed && mshgrp->member_cnt > 0) { + rd_kafka_mock_sharegroup_assignment_recalculate(mshgrp); + if (member) + MemberEpoch = member->member_epoch; + } + + mtx_unlock(&mcluster->lock); + } + + /* + * Build response + */ + + /* ThrottleTimeMs */ + rd_kafka_buf_write_i32(resp, 0); + + /* ErrorCode */ + rd_kafka_buf_write_i16(resp, err); + + /* ErrorMessage */ + if (err) + rd_kafka_buf_write_str(resp, rd_kafka_err2str(err), -1); + else + rd_kafka_buf_write_str(resp, NULL, -1); + + /* MemberId */ + if (!err && member) + rd_kafka_buf_write_str(resp, member->id, -1); + else + rd_kafka_buf_write_str(resp, NULL, -1); + + /* MemberEpoch */ + rd_kafka_buf_write_i32(resp, MemberEpoch); + + /* HeartbeatIntervalMs */ + if (mshgrp) + rd_kafka_buf_write_i32(resp, mshgrp->heartbeat_interval_ms); + else + rd_kafka_buf_write_i32(resp, 5000); + + /* Assignment */ + if (!err && member && member->assignment && + member->assignment->cnt > 0) { + rd_kafka_buf_write_i8(resp, 1); + rd_kafka_mock_handle_ShareGroupHeartbeat_write_TopicPartitions( + resp, member->assignment); + rd_kafka_buf_write_tags_empty(resp); + } else { + rd_kafka_buf_write_i8(resp, -1); + } + + rd_kafka_buf_write_tags_empty(resp); + + rd_kafka_mock_connection_send_response(mconn, resp); + + RD_IF_FREE(SubscribedTopicNames, rd_free); + return 0; + +err_parse: + RD_IF_FREE(SubscribedTopicNames, rd_free); + rd_kafka_buf_destroy(resp); + return -1; +} + /** * @brief Default request handlers */ @@ -3574,6 +3776,8 @@ const struct rd_kafka_mock_api_handler {2, 2, -1, rd_kafka_mock_handle_OffsetForLeaderEpoch}, [RD_KAFKAP_ConsumerGroupHeartbeat] = {1, 1, 1, rd_kafka_mock_handle_ConsumerGroupHeartbeat}, + [RD_KAFKAP_ShareGroupHeartbeat] = + {1, 1, 1, rd_kafka_mock_handle_ShareGroupHeartbeat}, [RD_KAFKAP_GetTelemetrySubscriptions] = {0, 0, 0, rd_kafka_mock_handle_GetTelemetrySubscriptions}, [RD_KAFKAP_PushTelemetry] = {0, 0, 0, diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index 78239a4aa7..06e4778c5a 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -261,6 +261,38 @@ typedef struct rd_kafka_mock_sgrp_s { int fetch_session_cnt; } rd_kafka_mock_sgrp_t; +/** + * @struct Share group (KIP-932). + */ +typedef struct rd_kafka_mock_sharegroup_s { + TAILQ_ENTRY(rd_kafka_mock_sharegroup_s) link; + struct rd_kafka_mock_cluster_s *cluster; /**< Cluster */ + char *id; /**< Share group Id */ + int32_t group_epoch; /**< Group epoch */ + int session_timeout_ms; /**< Session timeout */ + rd_kafka_timer_t session_tmr; /**< Session timeout timer */ + int heartbeat_interval_ms; /**< Heartbeat interval */ + TAILQ_HEAD(, rd_kafka_mock_sharegroup_member_s) + members; /**< Share group members */ + int member_cnt; /**< Number of share group members */ +} rd_kafka_mock_sharegroup_t; + +/** + * @struct Share group member (KIP-932). + */ +typedef struct rd_kafka_mock_sharegroup_member_s { + TAILQ_ENTRY(rd_kafka_mock_sharegroup_member_s) link; + char *id; /**< MemberId */ + rd_ts_t ts_last_activity; /**< Last heartbeat timestamp */ + int32_t member_epoch; /**< Current member epoch */ + int32_t previous_member_epoch; /**< Previous member epoch (allows + * client to catch up if response + * with bumped epoch was lost) */ + rd_list_t *subscribed_topic_names; /**< Subscribed topic names */ + rd_kafka_topic_partition_list_t *assignment; /**< Current assignment */ + struct rd_kafka_mock_connection_s *conn; /**< Connection */ + rd_kafka_mock_sharegroup_t *mshgrp; /**< Share group */ +} rd_kafka_mock_sharegroup_member_t; /** * @struct TransactionalId + PID (+ optional sequence state) @@ -513,6 +545,8 @@ struct rd_kafka_mock_cluster_s { TAILQ_HEAD(, rd_kafka_mock_sgrp_s) sgrps_share; + TAILQ_HEAD(, rd_kafka_mock_sharegroup_s) sharegrps; + /** Explicit coordinators (set with mock_set_coordinator()) */ TAILQ_HEAD(, rd_kafka_mock_coord_s) coords; @@ -551,6 +585,10 @@ struct rd_kafka_mock_cluster_s { int group_consumer_session_timeout_ms; /** Heartbeat interval (KIP 848) */ int group_consumer_heartbeat_interval_ms; + /** Session timeout (KIP 932) */ + int sharegroup_session_timeout_ms; + /** Heartbeat interval (KIP 932) */ + int sharegroup_heartbeat_interval_ms; } defaults; /**< Dynamic array of IO handlers for corresponding fd in .fds */ @@ -792,6 +830,43 @@ rd_kafka_mock_cgrp_consumer_member_t *rd_kafka_mock_cgrp_consumer_member_add( void rd_kafka_mock_cgrps_connection_closed(rd_kafka_mock_cluster_t *mcluster, rd_kafka_mock_connection_t *mconn); + +/* Share group (KIP-932) */ + +void rd_kafka_mock_sharegrps_init(rd_kafka_mock_cluster_t *mcluster); + +rd_kafka_mock_sharegroup_t * +rd_kafka_mock_sharegroup_find(rd_kafka_mock_cluster_t *mcluster, + const rd_kafkap_str_t *GroupId); + +rd_kafka_mock_sharegroup_t * +rd_kafka_mock_sharegroup_get(rd_kafka_mock_cluster_t *mcluster, + const rd_kafkap_str_t *GroupId); + +void rd_kafka_mock_sharegroup_destroy(rd_kafka_mock_sharegroup_t *mshgrp); + +rd_kafka_mock_sharegroup_member_t * +rd_kafka_mock_sharegroup_member_find(rd_kafka_mock_sharegroup_t *mshgrp, + const rd_kafkap_str_t *MemberId); + +void rd_kafka_mock_sharegroup_member_destroy( + rd_kafka_mock_sharegroup_t *mshgrp, + rd_kafka_mock_sharegroup_member_t *member); + +rd_kafka_mock_sharegroup_member_t * +rd_kafka_mock_sharegroup_member_get(rd_kafka_mock_sharegroup_t *mshgrp, + const rd_kafkap_str_t *MemberID, + int32_t MemberEpoch, + rd_kafka_mock_connection_t *mconn); + +rd_bool_t rd_kafka_mock_sharegroup_member_subscribed_topic_names_set( + rd_kafka_mock_sharegroup_member_t *member, + const rd_kafkap_str_t *SubscribedTopicNames, + int32_t SubscribedTopicNamesCnt); + +void rd_kafka_mock_sharegroup_assignment_recalculate( + rd_kafka_mock_sharegroup_t *mshgrp); + /** *@} */ diff --git a/src/rdkafka_mock_sharegrp.c b/src/rdkafka_mock_sharegrp.c new file mode 100644 index 0000000000..dc448ffb25 --- /dev/null +++ b/src/rdkafka_mock_sharegrp.c @@ -0,0 +1,440 @@ +/* + * librdkafka - Apache Kafka C library + * + * Copyright (c) 2025, Confluent Inc. + * 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. + */ + +/** + * Mocks + * + */ + + +#include "rdkafka_int.h" +#include "rdbuf.h" +#include "rdkafka_mock_int.h" + +/* Forward declaration - now declared in header */ + +/** + * @brief Initializes sharegroups in mock cluster + */ +void rd_kafka_mock_sharegrps_init(rd_kafka_mock_cluster_t *mcluster) { + TAILQ_INIT(&mcluster->sharegrps); + mcluster->defaults.sharegroup_session_timeout_ms = 45000; + mcluster->defaults.sharegroup_heartbeat_interval_ms = 5000; +} + +/** + * @brief Find a share group by GroupId. + */ +rd_kafka_mock_sharegroup_t * +rd_kafka_mock_sharegroup_find(rd_kafka_mock_cluster_t *mcluster, + const rd_kafkap_str_t *GroupId) { + rd_kafka_mock_sharegroup_t *mshgrp; + TAILQ_FOREACH(mshgrp, &mcluster->sharegrps, link) { + if (!rd_kafkap_str_cmp_str(GroupId, mshgrp->id)) + return mshgrp; + } + return NULL; +} + +/** + * @brief Get or create a share group + */ +rd_kafka_mock_sharegroup_t * +rd_kafka_mock_sharegroup_get(rd_kafka_mock_cluster_t *mcluster, + const rd_kafkap_str_t *GroupID) { + rd_kafka_mock_sharegroup_t *mshgrp; + + /* Check if the share group already exists */ + mshgrp = rd_kafka_mock_sharegroup_find(mcluster, GroupID); + if (mshgrp) + return mshgrp; + + /* Create new share group */ + mshgrp = rd_calloc(1, sizeof(*mshgrp)); + mshgrp->cluster = mcluster; + mshgrp->id = RD_KAFKAP_STR_DUP(GroupID); + mshgrp->group_epoch = 1; + mshgrp->session_timeout_ms = + mcluster->defaults.sharegroup_session_timeout_ms; + mshgrp->heartbeat_interval_ms = + mcluster->defaults.sharegroup_heartbeat_interval_ms; + + TAILQ_INIT(&mshgrp->members); + mshgrp->member_cnt = 0; + + TAILQ_INSERT_TAIL(&mcluster->sharegrps, mshgrp, link); + + return mshgrp; +} + +/** + * @brief Destroy a share group + */ +void rd_kafka_mock_sharegroup_destroy(rd_kafka_mock_sharegroup_t *mshgrp) { + rd_kafka_mock_sharegroup_member_t *member; + + TAILQ_REMOVE(&mshgrp->cluster->sharegrps, mshgrp, link); + + /* Destroy all members */ + while ((member = TAILQ_FIRST(&mshgrp->members))) + rd_kafka_mock_sharegroup_member_destroy(mshgrp, member); + + rd_free(mshgrp->id); + rd_free(mshgrp); +} + +/** + * @brief Find a share group member by MemberId. + */ +rd_kafka_mock_sharegroup_member_t * +rd_kafka_mock_sharegroup_member_find(rd_kafka_mock_sharegroup_t *mshgrp, + const rd_kafkap_str_t *MemberId) { + rd_kafka_mock_sharegroup_member_t *member; + TAILQ_FOREACH(member, &mshgrp->members, link) { + if (!rd_kafkap_str_cmp_str(MemberId, member->id)) + return member; + } + return NULL; +} + +/** + * @brief Destroy a share group member. + */ +void rd_kafka_mock_sharegroup_member_destroy( + rd_kafka_mock_sharegroup_t *mshgrp, + rd_kafka_mock_sharegroup_member_t *member) { + rd_assert(mshgrp->member_cnt > 0); + TAILQ_REMOVE(&mshgrp->members, member, link); + mshgrp->member_cnt--; + rd_free(member->id); + + RD_IF_FREE(member->subscribed_topic_names, rd_list_destroy_free); + RD_IF_FREE(member->assignment, rd_kafka_topic_partition_list_destroy); + rd_free(member); +} + +/** + * @brief Get or create a share group member. + */ +rd_kafka_mock_sharegroup_member_t * +rd_kafka_mock_sharegroup_member_get(rd_kafka_mock_sharegroup_t *mshgrp, + const rd_kafkap_str_t *MemberID, + int32_t MemberEpoch, + rd_kafka_mock_connection_t *mconn) { + rd_kafka_mock_sharegroup_member_t *member; + + /* Check if the member already exists */ + member = rd_kafka_mock_sharegroup_member_find(mshgrp, MemberID); + if (member) { + member->conn = mconn; + return member; + } + + /* Only create if epoch is 0 */ + if (MemberEpoch != 0) + return NULL; + + /* Create new member */ + member = rd_calloc(1, sizeof(*member)); + member->mshgrp = mshgrp; + member->id = RD_KAFKAP_STR_DUP(MemberID); + member->member_epoch = mshgrp->group_epoch; + member->previous_member_epoch = + -1; /* No previous epoch for new members */ + member->conn = mconn; + + TAILQ_INSERT_TAIL(&mshgrp->members, member, link); + mshgrp->member_cnt++; + + return member; +} + +/** + * @brief Update share group member's subscribed topic names. + */ +rd_bool_t rd_kafka_mock_sharegroup_member_subscribed_topic_names_set( + rd_kafka_mock_sharegroup_member_t *member, + const rd_kafkap_str_t *SubscribedTopicNames, + int32_t SubscribedTopicNamesCnt) { + rd_bool_t changed = rd_false; + int32_t i; + + if (!SubscribedTopicNamesCnt) { + /* No change */ + return rd_false; + } + + if (member->subscribed_topic_names) { + if (rd_list_cnt(member->subscribed_topic_names) == + SubscribedTopicNamesCnt) { + rd_bool_t same = rd_true; + char *topic; + int j; + + RD_LIST_FOREACH(topic, member->subscribed_topic_names, + j) { + rd_bool_t found = rd_false; + for (i = 0; i < SubscribedTopicNamesCnt; i++) { + if (!rd_kafkap_str_cmp_str( + &SubscribedTopicNames[i], + topic)) { + found = rd_true; + break; + } + } + if (!found) { + same = rd_false; + break; + } + } + if (same) + return rd_false; + } + } + + /* Subscription changed, update the list */ + changed = rd_true; + RD_IF_FREE(member->subscribed_topic_names, rd_list_destroy); + member->subscribed_topic_names = + rd_list_new(SubscribedTopicNamesCnt, rd_free); + + for (i = 0; i < SubscribedTopicNamesCnt; i++) { + rd_list_add(member->subscribed_topic_names, + RD_KAFKAP_STR_DUP(&SubscribedTopicNames[i])); + } + + return changed; +} + +/** + * @brief Collect all subscribed topic names from all members. + */ +static rd_list_t *rd_kafka_mock_sharegroup_collect_subscribed_topics( + rd_kafka_mock_sharegroup_t *mshgrp) { + rd_kafka_mock_sharegroup_member_t *member; + rd_list_t *all_topics; + + all_topics = rd_list_new(32, rd_free); + + TAILQ_FOREACH(member, &mshgrp->members, link) { + const char *topic; + int i; + + if (!member->subscribed_topic_names) + continue; + + RD_LIST_FOREACH(topic, member->subscribed_topic_names, i) { + const char *existing; + int j; + rd_bool_t found = rd_false; + + /* Check if topic already in all_topics */ + RD_LIST_FOREACH(existing, all_topics, j) { + if (!strcmp(topic, existing)) { + found = rd_true; + break; + } + } + + /* Add if not found */ + if (!found) { + rd_list_add(all_topics, rd_strdup(topic)); + } + } + } + + return all_topics; +} + +/** + * @brief Get list of member ID's subscribed to a topic. + */ +rd_list_t *rd_kafka_mock_sharegroup_get_members_for_topic( + rd_kafka_mock_sharegroup_t *mshgrp, + char *topic_name) { + rd_kafka_mock_sharegroup_member_t *member; + rd_list_t *subscribed_members; + int member_idx = 0; + + subscribed_members = rd_list_new(mshgrp->member_cnt, rd_free); + + TAILQ_FOREACH(member, &mshgrp->members, link) { + char *topic; + int i; + + if (member->subscribed_topic_names) { + RD_LIST_FOREACH(topic, member->subscribed_topic_names, + i) { + if (!strcmp(topic, topic_name)) { + int *idx = rd_malloc(sizeof(*idx)); + *idx = member_idx; + rd_list_add(subscribed_members, idx); + break; + } + } + } + member_idx++; + } + + return subscribed_members; +} + +/** + * @brief Assign partitions of a single topic to subscribed members. + */ +void rd_kafka_mock_sharegroup_assign_topic_partitions( + rd_kafka_mock_sharegroup_t *mshgrp, + rd_kafka_mock_topic_t *mtopic, + rd_list_t *subscribed_member_indices) { + int member_count; + int partition_cnt; + int partitions_per_member; + int extra_partitions; + int partition_idx; + int i; + + member_count = rd_list_cnt(subscribed_member_indices); + partition_cnt = mtopic->partition_cnt; + + if (member_count == 0 || partition_cnt == 0) + return; + + partitions_per_member = partition_cnt / member_count; + extra_partitions = partition_cnt % member_count; + partition_idx = 0; + + for (i = 0; i < member_count; i++) { + int *member_idx_ptr = + (int *)rd_list_elem(subscribed_member_indices, i); + rd_kafka_mock_sharegroup_member_t *member; + int j, cnt = 0; + int num_partitions; + + TAILQ_FOREACH(member, &mshgrp->members, link) { + if (cnt == *member_idx_ptr) + break; + cnt++; + } + + if (!member) + continue; + + num_partitions = + partitions_per_member + (i < extra_partitions ? 1 : 0); + + if (!member->assignment) + member->assignment = + rd_kafka_topic_partition_list_new(num_partitions); + + for (j = 0; j < num_partitions && partition_idx < partition_cnt; + j++, partition_idx++) { + rd_kafka_topic_partition_t *rktpar; + rktpar = rd_kafka_topic_partition_list_add( + member->assignment, mtopic->name, partition_idx); + /* Set topic ID so the response can include it */ + rd_kafka_topic_partition_set_topic_id(rktpar, + mtopic->id); + } + } +} + +/** + * @brief Recalculate assignments for all members in the share group. + */ +void rd_kafka_mock_sharegroup_assignment_recalculate( + rd_kafka_mock_sharegroup_t *mshgrp) { + rd_kafka_mock_sharegroup_member_t *member; + rd_list_t *all_topics; + char *topic_name; + int i; + + if (mshgrp->member_cnt == 0) + return; + + TAILQ_FOREACH(member, &mshgrp->members, link) { + if (member->assignment) { + rd_kafka_topic_partition_list_destroy( + member->assignment); + member->assignment = NULL; + } + } + + all_topics = rd_kafka_mock_sharegroup_collect_subscribed_topics(mshgrp); + + RD_LIST_FOREACH(topic_name, all_topics, i) { + rd_kafka_mock_topic_t *mtopic; + rd_list_t *subscribed_members; + + mtopic = rd_kafka_mock_topic_find(mshgrp->cluster, topic_name); + if (!mtopic) + continue; + + subscribed_members = + rd_kafka_mock_sharegroup_get_members_for_topic(mshgrp, + topic_name); + + rd_kafka_mock_sharegroup_assign_topic_partitions( + mshgrp, mtopic, subscribed_members); + + rd_list_destroy(subscribed_members); + } + + mshgrp->group_epoch++; + + TAILQ_FOREACH(member, &mshgrp->members, link) { + /* Save the current epoch as previous before bumping. + * This allows the client to catch up if the response + * with the new epoch was lost. */ + member->previous_member_epoch = member->member_epoch; + member->member_epoch = mshgrp->group_epoch; + } + + /* Print all member assignments */ + printf("\n=== SHARE GROUP ASSIGNMENT (epoch %d) ===\n", + mshgrp->group_epoch); + TAILQ_FOREACH(member, &mshgrp->members, link) { + printf(" %s -> ", member->id); + if (member->assignment && member->assignment->cnt > 0) { + int j; + for (j = 0; j < member->assignment->cnt; j++) { + rd_kafka_topic_partition_t *p = + &member->assignment->elems[j]; + printf("%s[%d]%s", p->topic, p->partition, + j < member->assignment->cnt - 1 ? ", " + : ""); + } + } else { + printf("(none)"); + } + printf("\n"); + } + printf("=========================================\n\n"); + fflush(stdout); + + rd_list_destroy(all_topics); +} diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index fd90f7aefb..e5b06498f8 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -145,6 +145,7 @@ set( 0152-rebootstrap.c 0153-memberid.c 0154-share-consumer.c + 0155-share_consumer_mock.c 8000-idle.cpp 8001-fetch_from_follower_mock_manual.c test.c diff --git a/win32/librdkafka.vcxproj b/win32/librdkafka.vcxproj index b31f895d62..9f30d0652a 100644 --- a/win32/librdkafka.vcxproj +++ b/win32/librdkafka.vcxproj @@ -237,6 +237,7 @@ + From ddab80ee654e46e523993c17af1eed7770e1da9b Mon Sep 17 00:00:00 2001 From: Ankith-Confluent Date: Tue, 3 Feb 2026 18:21:29 +0530 Subject: [PATCH 03/20] Add sharegroup session management and manual assignment features - Introduced functions to set session timeout and heartbeat interval for sharegroups. - Implemented manual target assignment for sharegroup members. - Enhanced connection handling to clear states for closed connections. --- src/rdkafka_mock.h | 38 ++++++ src/rdkafka_mock_cgrp.c | 1 + src/rdkafka_mock_handlers.c | 27 +++- src/rdkafka_mock_int.h | 9 +- src/rdkafka_mock_sharegrp.c | 241 +++++++++++++++++++++++++++++++++++- 5 files changed, 309 insertions(+), 7 deletions(-) diff --git a/src/rdkafka_mock.h b/src/rdkafka_mock.h index 8561206d0c..84dbb092c7 100644 --- a/src/rdkafka_mock.h +++ b/src/rdkafka_mock.h @@ -621,6 +621,44 @@ RD_EXPORT void rd_kafka_mock_set_group_consumer_heartbeat_interval_ms( rd_kafka_mock_cluster_t *mcluster, int group_consumer_heartbeat_interval_ms); +/** + * @brief Set the sharegroup session timeout in milliseconds. + * + * @param mcluster Mock cluster instance. + * @param session_timeout_ms Session timeout in milliseconds. + */ +RD_EXPORT void rd_kafka_mock_sharegroup_set_session_timeout( + rd_kafka_mock_cluster_t *mcluster, + int session_timeout_ms); + +/** + * @brief Set the sharegroup heartbeat interval in milliseconds. + * + * @param mcluster Mock cluster instance. + * @param heartbeat_interval_ms Heartbeat interval in milliseconds. + */ +RD_EXPORT void rd_kafka_mock_sharegroup_set_heartbeat_interval( + rd_kafka_mock_cluster_t *mcluster, + int heartbeat_interval_ms); + +/** + * @brief Set a manual target assignment for a sharegroup. + * + * This allows tests to override the automatic partition assignment + * and manually specify which partitions each member should get. + * + * @param mcluster Mock cluster instance. + * @param group_id The sharegroup ID. + * @param member_ids Array of member IDs (strings). + * @param assignments Array of partition lists (one per member). + * @param member_cnt Number of members (length of both arrays). + */ +RD_EXPORT void rd_kafka_mock_sharegroup_target_assignment( + rd_kafka_mock_cluster_t *mcluster, + const char *group_id, + const char **member_ids, + rd_kafka_topic_partition_list_t **assignments, + size_t member_cnt); /**@}*/ diff --git a/src/rdkafka_mock_cgrp.c b/src/rdkafka_mock_cgrp.c index bea8223097..c0154cd873 100644 --- a/src/rdkafka_mock_cgrp.c +++ b/src/rdkafka_mock_cgrp.c @@ -2015,4 +2015,5 @@ void rd_kafka_mock_cgrps_connection_closed(rd_kafka_mock_cluster_t *mcluster, rd_kafka_mock_connection_t *mconn) { rd_kafka_mock_cgrps_classic_connection_closed(mcluster, mconn); rd_kafka_mock_cgrps_consumer_connection_closed(mcluster, mconn); + rd_kafka_mock_sharegrps_connection_closed(mcluster, mconn); } diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 1fe1e76ea9..474b4b3a97 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -3571,7 +3571,7 @@ rd_kafka_mock_handle_ShareGroupHeartbeat(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_t *resp; rd_kafkap_str_t GroupId, MemberId, RackId; rd_kafkap_str_t *SubscribedTopicNames = NULL; - int32_t MemberEpoch, SubscribedTopicNamesCnt; + int32_t MemberEpoch = 0, SubscribedTopicNamesCnt; int32_t i; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; rd_kafka_mock_sharegroup_t *mshgrp = NULL; @@ -3580,9 +3580,30 @@ rd_kafka_mock_handle_ShareGroupHeartbeat(rd_kafka_mock_connection_t *mconn, resp = rd_kafka_mock_buf_new_response(rkbuf); + /* Inject Error */ + err = rd_kafka_mock_next_request_error(mconn, resp); + if (err) + goto build_response; + /* GroupId */ rd_kafka_buf_read_str(rkbuf, &GroupId); + /* Coordinator check */ + { + rd_kafka_mock_broker_t *mrkb; + + mrkb = rd_kafka_mock_cluster_get_coord( + mcluster, RD_KAFKA_COORD_GROUP, &GroupId); + + if (!mrkb) + err = RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE; + else if (mrkb != mconn->broker) + err = RD_KAFKA_RESP_ERR_NOT_COORDINATOR; + } + + if (err) + goto build_response; + /* MemberId */ rd_kafka_buf_read_str(rkbuf, &MemberId); @@ -3689,9 +3710,7 @@ rd_kafka_mock_handle_ShareGroupHeartbeat(rd_kafka_mock_connection_t *mconn, mtx_unlock(&mcluster->lock); } - /* - * Build response - */ +build_response: /* ThrottleTimeMs */ rd_kafka_buf_write_i32(resp, 0); diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index 06e4778c5a..fb6d59d809 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -273,8 +273,9 @@ typedef struct rd_kafka_mock_sharegroup_s { rd_kafka_timer_t session_tmr; /**< Session timeout timer */ int heartbeat_interval_ms; /**< Heartbeat interval */ TAILQ_HEAD(, rd_kafka_mock_sharegroup_member_s) - members; /**< Share group members */ - int member_cnt; /**< Number of share group members */ + members; /**< Share group members */ + int member_cnt; /**< Number of share group members */ + rd_bool_t manual_assignment; /**< Use manual assignment */ } rd_kafka_mock_sharegroup_t; /** @@ -867,6 +868,10 @@ rd_bool_t rd_kafka_mock_sharegroup_member_subscribed_topic_names_set( void rd_kafka_mock_sharegroup_assignment_recalculate( rd_kafka_mock_sharegroup_t *mshgrp); +void rd_kafka_mock_sharegrps_connection_closed( + rd_kafka_mock_cluster_t *mcluster, + rd_kafka_mock_connection_t *mconn); + /** *@} */ diff --git a/src/rdkafka_mock_sharegrp.c b/src/rdkafka_mock_sharegrp.c index dc448ffb25..a7851aa9b9 100644 --- a/src/rdkafka_mock_sharegrp.c +++ b/src/rdkafka_mock_sharegrp.c @@ -36,7 +36,17 @@ #include "rdbuf.h" #include "rdkafka_mock_int.h" -/* Forward declaration - now declared in header */ +/** + * @brief Share group target assignment (manual) + */ +typedef struct rd_kafka_mock_sharegroup_target_assignments_s { + rd_list_t member_ids; /**< List of member ids (char *) */ + rd_list_t assignment; /**< List of rd_kafka_topic_partition_list_t */ +} rd_kafka_mock_sharegroup_target_assignment_t; + +/* Forward declarations */ +static void rd_kafka_mock_sharegroup_session_tmr_cb(rd_kafka_timers_t *rkts, + void *arg); /** * @brief Initializes sharegroups in mock cluster @@ -87,6 +97,10 @@ rd_kafka_mock_sharegroup_get(rd_kafka_mock_cluster_t *mcluster, TAILQ_INIT(&mshgrp->members); mshgrp->member_cnt = 0; + rd_kafka_timer_start(&mcluster->timers, &mshgrp->session_tmr, + 1000 * 1000 /* 1s */, + rd_kafka_mock_sharegroup_session_tmr_cb, mshgrp); + TAILQ_INSERT_TAIL(&mcluster->sharegrps, mshgrp, link); return mshgrp; @@ -99,6 +113,8 @@ void rd_kafka_mock_sharegroup_destroy(rd_kafka_mock_sharegroup_t *mshgrp) { rd_kafka_mock_sharegroup_member_t *member; TAILQ_REMOVE(&mshgrp->cluster->sharegrps, mshgrp, link); + rd_kafka_timer_stop(&mshgrp->cluster->timers, &mshgrp->session_tmr, + RD_DO_LOCK); /* Destroy all members */ while ((member = TAILQ_FIRST(&mshgrp->members))) @@ -138,6 +154,57 @@ void rd_kafka_mock_sharegroup_member_destroy( rd_free(member); } +/** + * @brief Mark member as active. + */ +void rd_kafka_mock_sharegroup_member_active( + rd_kafka_mock_sharegroup_t *mshgrp, + rd_kafka_mock_sharegroup_member_t *member) { + rd_kafka_dbg(mshgrp->cluster->rk, MOCK, "MOCK", + "Marking mock share group member %s as active", + member->id); + member->ts_last_activity = rd_clock(); +} + +/** + * @brief Fence a member. + */ +void rd_kafka_mock_sharegroup_member_fenced( + rd_kafka_mock_sharegroup_t *mshgrp, + rd_kafka_mock_sharegroup_member_t *member) { + rd_kafka_dbg(mshgrp->cluster->rk, MOCK, "MOCK", + "Member %s is fenced from sharegroup %s", member->id, + mshgrp->id); + + rd_kafka_mock_sharegroup_member_destroy(mshgrp, member); +} + +/** + * @brief Check all members for inactivity and remove them if timed out. + */ +static void rd_kafka_mock_sharegroup_session_tmr_cb(rd_kafka_timers_t *rkts, + void *arg) { + rd_kafka_mock_sharegroup_t *mshgrp = arg; + rd_kafka_mock_sharegroup_member_t *member, *tmp; + rd_ts_t now = rd_clock(); + rd_kafka_mock_cluster_t *mcluster = mshgrp->cluster; + + mtx_lock(&mcluster->lock); + TAILQ_FOREACH_SAFE(member, &mshgrp->members, link, tmp) { + if (member->ts_last_activity + + (mshgrp->session_timeout_ms * 1000) > + now) + continue; + + rd_kafka_dbg(mcluster->rk, MOCK, "MOCK", + "Member %s session timed out for sharegroup %s", + member->id, mshgrp->id); + + rd_kafka_mock_sharegroup_member_fenced(mshgrp, member); + } + mtx_unlock(&mcluster->lock); +} + /** * @brief Get or create a share group member. */ @@ -152,6 +219,7 @@ rd_kafka_mock_sharegroup_member_get(rd_kafka_mock_sharegroup_t *mshgrp, member = rd_kafka_mock_sharegroup_member_find(mshgrp, MemberID); if (member) { member->conn = mconn; + rd_kafka_mock_sharegroup_member_active(mshgrp, member); return member; } @@ -170,6 +238,7 @@ rd_kafka_mock_sharegroup_member_get(rd_kafka_mock_sharegroup_t *mshgrp, TAILQ_INSERT_TAIL(&mshgrp->members, member, link); mshgrp->member_cnt++; + rd_kafka_mock_sharegroup_member_active(mshgrp, member); return member; } @@ -376,6 +445,10 @@ void rd_kafka_mock_sharegroup_assignment_recalculate( if (mshgrp->member_cnt == 0) return; + /* Skip automatic assignment if manual mode is enabled */ + if (mshgrp->manual_assignment) + return; + TAILQ_FOREACH(member, &mshgrp->members, link) { if (member->assignment) { rd_kafka_topic_partition_list_destroy( @@ -438,3 +511,169 @@ void rd_kafka_mock_sharegroup_assignment_recalculate( rd_list_destroy(all_topics); } + +/** + * @brief Create a new target assignment (manual) + */ +rd_kafka_mock_sharegroup_target_assignment_t * +rd_kafka_mock_sharegroup_target_assignment_new(void) { + rd_kafka_mock_sharegroup_target_assignment_t *target_assignment; + target_assignment = rd_calloc(1, sizeof(*target_assignment)); + rd_list_init(&target_assignment->member_ids, 0, rd_free); + rd_list_init(&target_assignment->assignment, 0, + (void *)rd_kafka_topic_partition_list_destroy); + + return target_assignment; +} + +/** + * @brief Destroy target assignment + */ +void rd_kafka_mock_sharegroup_target_assignment_destroy( + rd_kafka_mock_sharegroup_target_assignment_t *target_assignment) { + rd_list_destroy(&target_assignment->member_ids); + rd_list_destroy(&target_assignment->assignment); + rd_free(target_assignment); +} + +/** + * @brief Set the target assignment for the sharegroup. + * This applies the manual assignment to the members. + * + * @locks mcluster->lock MUST be held. + */ +static void rd_kafka_mock_sharegroup_target_assignment_set( + rd_kafka_mock_sharegroup_t *mshgrp, + rd_kafka_mock_sharegroup_target_assignment_t *target_assignment) { + rd_kafka_mock_sharegroup_member_t *member; + size_t i; + + for (i = 0; i < rd_list_cnt(&target_assignment->member_ids); i++) { + const char *member_id = + rd_list_elem(&target_assignment->member_ids, i); + const rd_kafka_topic_partition_list_t *partitions = + rd_list_elem(&target_assignment->assignment, i); + rd_kafkap_str_t *member_id_str; + + member_id_str = rd_kafkap_str_new(member_id, -1); + member = + rd_kafka_mock_sharegroup_member_find(mshgrp, member_id_str); + rd_kafkap_str_destroy(member_id_str); + + if (!member) { + rd_kafka_dbg(mshgrp->cluster->rk, MOCK, "MOCK", + "Cannot set target assignment for " + "non-existing member %s in sharegroup %s", + member_id, mshgrp->id); + continue; + } + + if (member->assignment) { + rd_kafka_topic_partition_list_destroy( + member->assignment); + } + + member->assignment = + rd_kafka_topic_partition_list_copy(partitions); + + rd_kafka_dbg( + mshgrp->cluster->rk, MOCK, "MOCK", + "Target assignment set for member %s: %d partition(s)", + member_id, member->assignment->cnt); + } + + /* Bump the epochs */ + TAILQ_FOREACH(member, &mshgrp->members, link) { + member->previous_member_epoch = member->member_epoch; + member->member_epoch = ++mshgrp->group_epoch; + } +} + +/** + * @brief Manual target assignment interface for sharegroups. + */ +void rd_kafka_mock_sharegroup_target_assignment( + rd_kafka_mock_cluster_t *mcluster, + const char *group_id, + const char **member_ids, + rd_kafka_topic_partition_list_t **assignment, + size_t member_cnt) { + rd_kafka_mock_sharegroup_t *mshgrp; + rd_kafka_mock_sharegroup_target_assignment_t *target_assignment; + size_t i; + rd_kafkap_str_t *group_id_str; + + mtx_lock(&mcluster->lock); + group_id_str = rd_kafkap_str_new(group_id, -1); + mshgrp = rd_kafka_mock_sharegroup_find(mcluster, group_id_str); + rd_kafkap_str_destroy(group_id_str); + + if (!mshgrp) { + rd_kafka_log(mcluster->rk, LOG_ERR, "MOCK", + "Sharegroup %s not found for target assignment", + group_id); + mtx_unlock(&mcluster->lock); + return; + } + + mshgrp->manual_assignment = rd_true; + target_assignment = rd_kafka_mock_sharegroup_target_assignment_new(); + + for (i = 0; i < member_cnt; i++) { + rd_list_add(&target_assignment->member_ids, + rd_strdup(member_ids[i])); + rd_list_add(&target_assignment->assignment, + rd_kafka_topic_partition_list_copy(assignment[i])); + } + rd_kafka_mock_sharegroup_target_assignment_set(mshgrp, + target_assignment); + rd_kafka_mock_sharegroup_target_assignment_destroy(target_assignment); + mtx_unlock(&mcluster->lock); +} + +/** + * @brief Set the sharegroup session timeout for the sharegroup. + */ +void rd_kafka_mock_sharegroup_set_session_timeout( + rd_kafka_mock_cluster_t *mcluster, + int session_timeout_ms) { + mtx_lock(&mcluster->lock); + mcluster->defaults.sharegroup_session_timeout_ms = session_timeout_ms; + mtx_unlock(&mcluster->lock); +} + +/** + * @brief Set the sharegroup heartbeat interval for the sharegroup. + */ +void rd_kafka_mock_sharegroup_set_heartbeat_interval( + rd_kafka_mock_cluster_t *mcluster, + int heartbeat_interval_ms) { + mtx_lock(&mcluster->lock); + mcluster->defaults.sharegroup_heartbeat_interval_ms = + heartbeat_interval_ms; + mtx_unlock(&mcluster->lock); +} + +/** + * @brief A client connection closed, check if any sharegroup has any + * state for this connection that needs to be cleared. + * + * @param mcluster Cluster to search in. + * @param mconn Connection that was closed. + * + * @locks mcluster->lock MUST be held. + */ +void rd_kafka_mock_sharegrps_connection_closed( + rd_kafka_mock_cluster_t *mcluster, + rd_kafka_mock_connection_t *mconn) { + rd_kafka_mock_sharegroup_t *mshgrp; + + TAILQ_FOREACH(mshgrp, &mcluster->sharegrps, link) { + rd_kafka_mock_sharegroup_member_t *member; + TAILQ_FOREACH(member, &mshgrp->members, link) { + if (member->conn == mconn) { + member->conn = NULL; + } + } + } +} \ No newline at end of file From 26a66e530babdbae222a0f6781637313a2e1f721 Mon Sep 17 00:00:00 2001 From: Ankith-Confluent Date: Wed, 4 Feb 2026 21:48:25 +0530 Subject: [PATCH 04/20] Add function to retrieve member IDs from a sharegroup and update assignment handling --- src/rdkafka_mock.h | 18 +++++++++ src/rdkafka_mock_handlers.c | 6 ++- src/rdkafka_mock_sharegrp.c | 73 +++++++++++++++++++++++++++++++++++++ 3 files changed, 95 insertions(+), 2 deletions(-) diff --git a/src/rdkafka_mock.h b/src/rdkafka_mock.h index 84dbb092c7..18eb030770 100644 --- a/src/rdkafka_mock.h +++ b/src/rdkafka_mock.h @@ -660,6 +660,24 @@ RD_EXPORT void rd_kafka_mock_sharegroup_target_assignment( rd_kafka_topic_partition_list_t **assignments, size_t member_cnt); +/** + * @brief Retrieve the member IDs from a sharegroup. + * + * @param mcluster Mock cluster instance. + * @param group_id The sharegroup ID. + * @param member_ids_out Output array of member IDs. Caller must free each + * string with rd_free() and the array itself. + * @param member_cnt_out Output count of members. + * + * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success, + * RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND if sharegroup not found. + */ +RD_EXPORT rd_kafka_resp_err_t rd_kafka_mock_sharegroup_get_member_ids( + rd_kafka_mock_cluster_t *mcluster, + const char *group_id, + char ***member_ids_out, + size_t *member_cnt_out); + /**@}*/ #ifdef __cplusplus diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 474b4b3a97..aecb34a90a 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -3740,8 +3740,10 @@ rd_kafka_mock_handle_ShareGroupHeartbeat(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_write_i32(resp, 5000); /* Assignment */ - if (!err && member && member->assignment && - member->assignment->cnt > 0) { + if (!err && member && member->assignment) { + /* Send assignment even if empty (cnt == 0). + * Null (-1) means "no change", while an empty assignment + * means "you have 0 partitions". */ rd_kafka_buf_write_i8(resp, 1); rd_kafka_mock_handle_ShareGroupHeartbeat_write_TopicPartitions( resp, member->assignment); diff --git a/src/rdkafka_mock_sharegrp.c b/src/rdkafka_mock_sharegrp.c index a7851aa9b9..336d3b1bc9 100644 --- a/src/rdkafka_mock_sharegrp.c +++ b/src/rdkafka_mock_sharegrp.c @@ -576,6 +576,26 @@ static void rd_kafka_mock_sharegroup_target_assignment_set( member->assignment = rd_kafka_topic_partition_list_copy(partitions); + /* Set topic IDs on each partition so the heartbeat response + * can include them (ShareGroupHeartbeat uses topic IDs) */ + { + int j; + for (j = 0; j < member->assignment->cnt; j++) { + rd_kafka_topic_partition_t *rktpar = + &member->assignment->elems[j]; + rd_kafkap_str_t topic_str = { + .str = rktpar->topic, + .len = strlen(rktpar->topic)}; + rd_kafka_mock_topic_t *mtopic = + rd_kafka_mock_topic_find_by_kstr( + mshgrp->cluster, &topic_str); + if (mtopic) { + rd_kafka_topic_partition_set_topic_id( + rktpar, mtopic->id); + } + } + } + rd_kafka_dbg( mshgrp->cluster->rk, MOCK, "MOCK", "Target assignment set for member %s: %d partition(s)", @@ -676,4 +696,57 @@ void rd_kafka_mock_sharegrps_connection_closed( } } } +} + +/** + * @brief Retrieve the member IDs from a sharegroup. + * + * @param mcluster Mock cluster instance. + * @param group_id The sharegroup ID. + * @param member_ids_out Output array of member IDs (caller must free each + * string and the array itself). + * @param member_cnt_out Output count of members. + * + * @returns RD_KAFKA_RESP_ERR_NO_ERROR on success, + * RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND if sharegroup not found. + */ +rd_kafka_resp_err_t rd_kafka_mock_sharegroup_get_member_ids( + rd_kafka_mock_cluster_t *mcluster, + const char *group_id, + char ***member_ids_out, + size_t *member_cnt_out) { + rd_kafka_mock_sharegroup_t *mshgrp; + rd_kafka_mock_sharegroup_member_t *member; + rd_kafkap_str_t *group_id_str; + char **member_ids; + size_t i; + + mtx_lock(&mcluster->lock); + group_id_str = rd_kafkap_str_new(group_id, -1); + mshgrp = rd_kafka_mock_sharegroup_find(mcluster, group_id_str); + rd_kafkap_str_destroy(group_id_str); + + if (!mshgrp) { + mtx_unlock(&mcluster->lock); + *member_ids_out = NULL; + *member_cnt_out = 0; + return RD_KAFKA_RESP_ERR_GROUP_ID_NOT_FOUND; + } + + *member_cnt_out = mshgrp->member_cnt; + if (mshgrp->member_cnt == 0) { + mtx_unlock(&mcluster->lock); + *member_ids_out = NULL; + return RD_KAFKA_RESP_ERR_NO_ERROR; + } + + member_ids = rd_malloc(sizeof(*member_ids) * mshgrp->member_cnt); + i = 0; + TAILQ_FOREACH(member, &mshgrp->members, link) { + member_ids[i++] = rd_strdup(member->id); + } + + mtx_unlock(&mcluster->lock); + *member_ids_out = member_ids; + return RD_KAFKA_RESP_ERR_NO_ERROR; } \ No newline at end of file From b1a6f0cd333c2ed3778a728990cfa48b135c6df6 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Thu, 5 Feb 2026 17:28:41 +0530 Subject: [PATCH 05/20] Mock Broker : Implement share fetch --- src/rdkafka_mock.c | 5 + src/rdkafka_mock_handlers.c | 610 +++++++++++++++++++++++++++++++++++- src/rdkafka_mock_int.h | 83 +++++ 3 files changed, 697 insertions(+), 1 deletion(-) diff --git a/src/rdkafka_mock.c b/src/rdkafka_mock.c index 3c8b00988f..0bd13539a8 100644 --- a/src/rdkafka_mock.c +++ b/src/rdkafka_mock.c @@ -2778,6 +2778,7 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) { rd_kafka_mock_cgrp_consumer_t *mcgrp_consumer; rd_kafka_mock_sgrp_t *msgrp; rd_kafka_mock_sharegroup_t *mshgrp; + rd_kafka_mock_sgrp_t *msgrp; rd_kafka_mock_coord_t *mcoord; rd_kafka_mock_error_stack_t *errstack; thrd_t dummy_rkb_thread; @@ -2802,6 +2803,9 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) { while ((mshgrp = TAILQ_FIRST(&mcluster->sharegrps))) rd_kafka_mock_sharegroup_destroy(mshgrp); + while ((msgrp = TAILQ_FIRST(&mcluster->sgrps_share))) + rd_kafka_mock_sgrp_destroy(msgrp); + while ((mcoord = TAILQ_FIRST(&mcluster->coords))) rd_kafka_mock_coord_destroy(mcluster, mcoord); @@ -2922,6 +2926,7 @@ rd_kafka_mock_cluster_t *rd_kafka_mock_cluster_new(rd_kafka_t *rk, TAILQ_INIT(&mcluster->sgrps_share); rd_kafka_mock_sharegrps_init(mcluster); + TAILQ_INIT(&mcluster->sgrps_share); TAILQ_INIT(&mcluster->coords); diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index aecb34a90a..5efce2147a 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -1461,7 +1461,26 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_write_i32(resp, INT32_MIN); } - rd_kafka_buf_skip_tags(rkbuf); + /* ForgottenToppars */ + { + int32_t ForgottenTopicsCnt; + rd_kafka_buf_read_arraycnt(rkbuf, &ForgottenTopicsCnt, + RD_KAFKAP_TOPICS_MAX); + while (ForgottenTopicsCnt-- > 0) { + rd_kafka_Uuid_t TopicId = RD_KAFKA_UUID_ZERO; + int32_t PartitionCnt; + rd_kafka_buf_read_uuid(rkbuf, &TopicId); + rd_kafka_buf_read_arraycnt(rkbuf, &PartitionCnt, + RD_KAFKAP_PARTITIONS_MAX); + while (PartitionCnt-- > 0) { + rd_kafka_buf_read_i32(rkbuf, NULL); /* Partition */ + } + rd_kafka_buf_skip_tags(rkbuf); /* Topic tags */ + } + } + + if (rd_kafka_buf_read_remain(rkbuf) > 0) + rd_kafka_buf_skip_tags(rkbuf); rd_kafka_buf_write_tags_empty(resp); if (requested_topics) @@ -3765,6 +3784,595 @@ rd_kafka_mock_handle_ShareGroupHeartbeat(rd_kafka_mock_connection_t *mconn, return -1; } +/** + * @brief Handle ShareFetch (KIP-932) - request parsing only. + */ +static rd_bool_t +rd_kafka_mock_tplist_equal_by_id(rd_kafka_topic_partition_list_t *a, + rd_kafka_topic_partition_list_t *b) { + int i; + + if (!a && !b) + return rd_true; + if (!a || !b) + return rd_false; + if (a->cnt != b->cnt) + return rd_false; + + rd_kafka_topic_partition_list_sort_by_topic_id(a); + rd_kafka_topic_partition_list_sort_by_topic_id(b); + + for (i = 0; i < a->cnt; i++) { + const rd_kafka_topic_partition_t *pa = &a->elems[i]; + const rd_kafka_topic_partition_t *pb = &b->elems[i]; + rd_kafka_Uuid_t ta = + rd_kafka_topic_partition_get_topic_id(pa); + rd_kafka_Uuid_t tb = + rd_kafka_topic_partition_get_topic_id(pb); + + if (pa->partition != pb->partition) + return rd_false; + if (rd_kafka_Uuid_cmp(ta, tb) != 0) + return rd_false; + } + + return rd_true; +} + +static rd_kafka_mock_sgrp_partmeta_t * +rd_kafka_mock_sgrp_partmeta_find(rd_kafka_mock_sgrp_t *sgrp, + rd_kafka_Uuid_t topic_id, + int32_t partition) { + rd_kafka_mock_sgrp_partmeta_t *pmeta; + + TAILQ_FOREACH(pmeta, &sgrp->partitions, link) { + if (pmeta->partition != partition) + continue; + if (!rd_kafka_Uuid_cmp(pmeta->topic_id, topic_id)) + return pmeta; + } + + return NULL; +} + +static rd_kafka_mock_sgrp_partmeta_t * +rd_kafka_mock_sgrp_partmeta_get(rd_kafka_mock_sgrp_t *sgrp, + rd_kafka_Uuid_t topic_id, + int32_t partition, + const rd_kafka_mock_partition_t *mpart) { + rd_kafka_mock_sgrp_partmeta_t *pmeta; + int64_t log_start; + int64_t log_end; + + pmeta = rd_kafka_mock_sgrp_partmeta_find(sgrp, topic_id, partition); + if (pmeta) { + log_start = mpart->start_offset; + log_end = mpart->end_offset; + if (log_start > pmeta->spso) + pmeta->spso = log_start; + if (log_end > log_start) { + int64_t new_speo = log_end - 1; + if (new_speo > pmeta->speo) + pmeta->speo = new_speo; + } + return pmeta; + } + + pmeta = rd_calloc(1, sizeof(*pmeta)); + pmeta->topic_id = topic_id; + pmeta->partition = partition; + pmeta->spso = mpart->start_offset; + if (mpart->end_offset > mpart->start_offset) + pmeta->speo = mpart->end_offset - 1; + else + pmeta->speo = mpart->start_offset - 1; + TAILQ_INIT(&pmeta->inflight); + + TAILQ_INSERT_TAIL(&sgrp->partitions, pmeta, link); + sgrp->partition_cnt++; + + return pmeta; +} + +static rd_kafka_mock_sgrp_record_state_t * +rd_kafka_mock_sgrp_record_state_find(rd_kafka_mock_sgrp_partmeta_t *pmeta, + int64_t offset) { + rd_kafka_mock_sgrp_record_state_t *state; + + TAILQ_FOREACH(state, &pmeta->inflight, link) { + if (state->offset == offset) + return state; + } + + return NULL; +} + +static rd_kafka_mock_sgrp_record_state_t * +rd_kafka_mock_sgrp_record_state_get(rd_kafka_mock_sgrp_partmeta_t *pmeta, + int64_t offset) { + rd_kafka_mock_sgrp_record_state_t *state; + + state = rd_kafka_mock_sgrp_record_state_find(pmeta, offset); + if (state) + return state; + + state = rd_calloc(1, sizeof(*state)); + state->offset = offset; + state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE; + TAILQ_INSERT_TAIL(&pmeta->inflight, state, link); + pmeta->inflight_cnt++; + + return state; +} + +static int32_t +rd_kafka_mock_msgset_est_record_size(const rd_kafka_mock_msgset_t *mset) { + int64_t record_cnt = mset->last_offset - mset->first_offset + 1; + int32_t size; + + if (record_cnt <= 0) + return 1; + + size = (int32_t)(RD_KAFKAP_BYTES_LEN(&mset->bytes) / record_cnt); + if (size <= 0) + size = 1; + + return size; +} + +static void +rd_kafka_mock_sgrp_acquire_available_offsets( + rd_kafka_mock_sgrp_partmeta_t *pmeta, + const rd_kafka_mock_partition_t *mpart, + const rd_kafkap_str_t *member_id, + rd_ts_t lock_expiry_ts, + int64_t *remaining_records, + int64_t *remaining_bytes, + int *acquired_cnt, + int64_t *acquired_bytes) { + int64_t offset; + + for (offset = pmeta->spso; offset <= pmeta->speo; offset++) { + const rd_kafka_mock_msgset_t *mset; + rd_kafka_mock_sgrp_record_state_t *state; + int32_t est_size; + + if (remaining_records && *remaining_records == 0) + break; + if (remaining_bytes && *remaining_bytes == 0) + break; + + state = rd_kafka_mock_sgrp_record_state_find(pmeta, offset); + if (state && + state->state != RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE) + continue; + + mset = rd_kafka_mock_msgset_find(mpart, offset, rd_false); + if (!mset) + continue; + + est_size = rd_kafka_mock_msgset_est_record_size(mset); + if (remaining_bytes && *remaining_bytes > 0 && + est_size > *remaining_bytes) + break; + + state = rd_kafka_mock_sgrp_record_state_get(pmeta, offset); + state->state = RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED; + state->lock_expiry_ts = lock_expiry_ts; + state->delivery_count++; + RD_IF_FREE(state->owner_member_id, rd_free); + state->owner_member_id = RD_KAFKAP_STR_DUP(member_id); + + (*acquired_cnt)++; + *acquired_bytes += est_size; + if (remaining_records && *remaining_records > 0) + (*remaining_records)--; + if (remaining_bytes && *remaining_bytes > 0) + (*remaining_bytes) -= est_size; + } +} + +static void +rd_kafka_mock_sgrp_partmeta_prune_archived(rd_kafka_mock_sgrp_partmeta_t *pmeta) { + rd_kafka_mock_sgrp_record_state_t *state, *tmp; + + TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { + if (state->state != RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED) + continue; + if (state->offset <= pmeta->speo) + continue; + + TAILQ_REMOVE(&pmeta->inflight, state, link); + pmeta->inflight_cnt--; + rd_free(state->owner_member_id); + rd_free(state); + } +} + +static const rd_kafka_mock_msgset_t * +rd_kafka_mock_sgrp_first_acquired_msgset( + rd_kafka_mock_sgrp_partmeta_t *pmeta, + const rd_kafka_mock_partition_t *mpart, + const rd_kafkap_str_t *member_id, + rd_ts_t now) { + int64_t offset; + + for (offset = pmeta->spso; offset <= pmeta->speo; offset++) { + rd_kafka_mock_sgrp_record_state_t *state = + rd_kafka_mock_sgrp_record_state_find(pmeta, offset); + const rd_kafka_mock_msgset_t *mset; + + if (!state || + state->state != RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) + continue; + + if (state->lock_expiry_ts && + state->lock_expiry_ts <= now) + continue; + + if (rd_kafkap_str_cmp_str(member_id, state->owner_member_id)) + continue; + + mset = rd_kafka_mock_msgset_find(mpart, offset, rd_false); + if (mset) + return mset; + } + + return NULL; +} + +static int +rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, + rd_kafka_buf_t *rkbuf) { + const rd_bool_t log_decode_errors = rd_true; + rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster; + rd_kafka_buf_t *resp = rd_kafka_mock_buf_new_response(rkbuf); + rd_kafkap_str_t GroupId, MemberId; + int32_t SessionId = -1, SessionEpoch = -1, MaxWaitMs = 0, MinBytes = 0, + MaxBytes = 0, MaxRecords = 0, BatchSize = 0; + int32_t TopicsCnt; + rd_kafka_topic_partition_list_t *requested_partitions = NULL; + rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; + rd_kafka_mock_sgrp_t *sgrp = NULL; + rd_kafka_mock_sgrp_fetch_session_t *session = NULL; + static int32_t next_session_id = 1; + + (void)log_decode_errors; + + rd_kafka_buf_read_str(rkbuf, &GroupId); + rd_kafka_buf_read_str(rkbuf, &MemberId); + /* Fetcher sends SessionEpoch only (no SessionId) */ + rd_kafka_buf_read_i32(rkbuf, &SessionEpoch); + rd_kafka_buf_read_i32(rkbuf, &MaxWaitMs); + rd_kafka_buf_read_i32(rkbuf, &MinBytes); + rd_kafka_buf_read_i32(rkbuf, &MaxBytes); + rd_kafka_buf_read_i32(rkbuf, &MaxRecords); + rd_kafka_buf_read_i32(rkbuf, &BatchSize); + + requested_partitions = rd_kafka_topic_partition_list_new(0); + + rd_kafka_buf_read_arraycnt(rkbuf, &TopicsCnt, RD_KAFKAP_TOPICS_MAX); + while (TopicsCnt-- > 0) { + rd_kafka_Uuid_t TopicId = RD_KAFKA_UUID_ZERO; + int32_t PartitionCnt; + + rd_kafka_buf_read_uuid(rkbuf, &TopicId); + rd_kafka_buf_read_arraycnt(rkbuf, &PartitionCnt, + RD_KAFKAP_PARTITIONS_MAX); + + while (PartitionCnt-- > 0) { + int32_t Partition; + int32_t AckBatchCnt; + rd_kafka_topic_partition_t *rktpar; + + rd_kafka_buf_read_i32(rkbuf, &Partition); + rd_kafka_buf_read_arraycnt(rkbuf, &AckBatchCnt, -1); + while (AckBatchCnt-- > 0) { + int32_t AckTypeCnt; + int8_t AckType; + rd_kafka_buf_read_i64(rkbuf, NULL); /* FirstOffset */ + rd_kafka_buf_read_i64(rkbuf, NULL); /* LastOffset */ + rd_kafka_buf_read_arraycnt(rkbuf, &AckTypeCnt, + -1); + while (AckTypeCnt-- > 0) { + rd_kafka_buf_read_i8(rkbuf, &AckType); + (void)AckType; + } + rd_kafka_buf_skip_tags(rkbuf); + } + + rktpar = rd_kafka_topic_partition_list_add( + requested_partitions, "", Partition); + rd_kafka_topic_partition_set_topic_id(rktpar, TopicId); + + rd_kafka_buf_skip_tags(rkbuf); + } + + rd_kafka_buf_skip_tags(rkbuf); + } + + rd_kafka_buf_skip_tags(rkbuf); + + rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK", + "ShareFetch parsed: group %.*s member %.*s " + "session_id %" PRId32 " session_epoch %" PRId32 + " max_wait %" PRId32 " min_bytes %" PRId32 + " max_bytes %" PRId32 " max_records %" PRId32 + " batch_size %" PRId32, + RD_KAFKAP_STR_PR(&GroupId), RD_KAFKAP_STR_PR(&MemberId), + SessionId, SessionEpoch, MaxWaitMs, MinBytes, MaxBytes, + MaxRecords, BatchSize); + + err = rd_kafka_mock_next_request_error(mconn, resp); + + if (!err) { + int64_t remaining_records = + MaxRecords > 0 ? (int64_t)MaxRecords : -1; + int64_t remaining_bytes = + MaxBytes > 0 ? (int64_t)MaxBytes : -1; + int acquired_cnt = 0; + int64_t acquired_bytes = 0; + rd_ts_t now = rd_clock(); + + // Handle ShareFetch session management + mtx_lock(&mcluster->lock); + sgrp = rd_kafka_mock_sgrp_get(mcluster, &GroupId); + + if (SessionId < 0) { + TAILQ_FOREACH(session, &sgrp->fetch_sessions, link) { + if (!rd_kafkap_str_cmp_str(&MemberId, + session->member_id)) + break; + } + if (!session) { + session = rd_calloc(1, sizeof(*session)); + session->member_id = RD_KAFKAP_STR_DUP(&MemberId); + session->session_id = next_session_id++; + session->session_epoch = 0; + session->ts_last_activity = rd_clock(); + session->partitions = + rd_kafka_topic_partition_list_copy( + requested_partitions); + TAILQ_INSERT_TAIL(&sgrp->fetch_sessions, session, + link); + sgrp->fetch_session_cnt++; + } + } else { + TAILQ_FOREACH(session, &sgrp->fetch_sessions, link) { + if (session->session_id != SessionId) + continue; + if (rd_kafkap_str_cmp_str(&MemberId, + session->member_id)) + continue; + break; + } + } + + if (!session) { + err = RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH; + } else if (SessionEpoch != -1 && + SessionEpoch != session->session_epoch) { + err = RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH; + } else { + if (!rd_kafka_mock_tplist_equal_by_id( + requested_partitions, session->partitions)) { + session->session_epoch++; + RD_IF_FREE( + session->partitions, + rd_kafka_topic_partition_list_destroy); + session->partitions = + rd_kafka_topic_partition_list_copy( + requested_partitions); + } + session->ts_last_activity = rd_clock(); + } + + if (!err && sgrp) { + rd_kafka_topic_partition_t *rktpar; + RD_KAFKA_TPLIST_FOREACH(rktpar, requested_partitions) { + rd_kafka_Uuid_t topic_id = + rd_kafka_topic_partition_get_topic_id( + rktpar); + rd_kafka_mock_topic_t *mtopic = + rd_kafka_mock_topic_find_by_id(mcluster, + topic_id); + rd_kafka_mock_partition_t *mpart; + + if (!mtopic) { + err = + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; + break; + } + + mpart = rd_kafka_mock_partition_find( + mtopic, rktpar->partition); + if (!mpart) { + err = + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; + break; + } + + rd_kafka_mock_sgrp_partmeta_t *pmeta = + rd_kafka_mock_sgrp_partmeta_get( + sgrp, topic_id, rktpar->partition, + mpart); + rd_kafka_mock_sgrp_partmeta_prune_archived( + pmeta); + rd_kafka_mock_sgrp_acquire_available_offsets( + pmeta, mpart, + &MemberId, + now + + (sgrp->session_timeout_ms * 1000), + MaxRecords > 0 ? &remaining_records : NULL, + MaxBytes > 0 ? &remaining_bytes : NULL, + &acquired_cnt, &acquired_bytes); + } + } + + rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK", + "ShareFetch acquired: %d records, %" PRId64 + " bytes", + acquired_cnt, acquired_bytes); + + /* Response: ThrottleTimeMs */ + rd_kafka_buf_write_i32(resp, 0); + /* Response: ErrorCode */ + rd_kafka_buf_write_i16(resp, err); + /* Response: ErrorMessage */ + if (err) + rd_kafka_buf_write_str(resp, rd_kafka_err2str(err), -1); + else + rd_kafka_buf_write_str(resp, NULL, -1); + /* Response: AcquisitionLockTimeoutMs */ + rd_kafka_buf_write_i32(resp, sgrp ? sgrp->session_timeout_ms : 0); + + rd_kafka_topic_partition_list_sort_by_topic_id( + requested_partitions); + + { + int i = 0; + int topic_cnt = 0; + rd_kafka_Uuid_t current_topic = RD_KAFKA_UUID_ZERO; + + for (i = 0; i < requested_partitions->cnt; i++) { + rd_kafka_topic_partition_t *rktpar = + &requested_partitions->elems[i]; + rd_kafka_Uuid_t topic_id = + rd_kafka_topic_partition_get_topic_id( + rktpar); + if (i == 0 || + rd_kafka_Uuid_cmp(topic_id, + current_topic) != 0) { + topic_cnt++; + current_topic = topic_id; + } + } + + /* Response: #Topics */ + rd_kafka_buf_write_arraycnt(resp, topic_cnt); + + i = 0; + while (i < requested_partitions->cnt) { + int j; + rd_kafka_Uuid_t topic_id = + rd_kafka_topic_partition_get_topic_id( + &requested_partitions->elems[i]); + int part_cnt = 0; + + for (j = i; j < requested_partitions->cnt; + j++) { + rd_kafka_Uuid_t next_topic_id = + rd_kafka_topic_partition_get_topic_id( + &requested_partitions->elems[j]); + if (rd_kafka_Uuid_cmp(topic_id, + next_topic_id) != + 0) + break; + part_cnt++; + } + + /* Response: TopicId */ + rd_kafka_buf_write_uuid(resp, &topic_id); + /* Response: #Partitions */ + rd_kafka_buf_write_arraycnt(resp, part_cnt); + + for (j = i; j < i + part_cnt; j++) { + rd_kafka_topic_partition_t *rktpar = + &requested_partitions->elems[j]; + rd_kafka_mock_topic_t *mtopic = + rd_kafka_mock_topic_find_by_id( + mcluster, topic_id); + rd_kafka_mock_partition_t *mpart = + mtopic ? rd_kafka_mock_partition_find( + mtopic, + rktpar->partition) + : NULL; + rd_kafka_mock_sgrp_partmeta_t *pmeta = + mpart + ? rd_kafka_mock_sgrp_partmeta_find( + sgrp, topic_id, + rktpar->partition) + : NULL; + const rd_kafka_mock_msgset_t *mset = + (mpart && pmeta) + ? rd_kafka_mock_sgrp_first_acquired_msgset( + pmeta, mpart, &MemberId, + now) + : NULL; + rd_kafka_resp_err_t part_err = + mpart ? RD_KAFKA_RESP_ERR_NO_ERROR + : RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; + + /* Response: Partition */ + rd_kafka_buf_write_i32( + resp, rktpar->partition); + /* Response: PartitionFetchErrorCode */ + rd_kafka_buf_write_i16(resp, part_err); + /* Response: PartitionFetchErrorString */ + if (part_err) + rd_kafka_buf_write_str( + resp, + rd_kafka_err2str(part_err), + -1); + else + rd_kafka_buf_write_str( + resp, NULL, -1); + /* Response: AcknowledgementErrorCode */ + rd_kafka_buf_write_i16(resp, 0); + /* Response: AcknowledgementErrorString */ + rd_kafka_buf_write_str(resp, NULL, -1); + /* Response: CurrentLeader */ + rd_kafka_buf_write_i32(resp, -1); + rd_kafka_buf_write_i32(resp, -1); + rd_kafka_buf_write_tags_empty(resp); + /* Response: Records */ + rd_kafka_buf_write_kbytes( + resp, mset ? &mset->bytes : NULL); + /* Response: AcquiredRecords */ + rd_kafka_buf_write_arraycnt(resp, 0); + /* Response: Partition tags */ + rd_kafka_buf_write_tags_empty(resp); + } + + /* Response: Topic tags */ + rd_kafka_buf_write_tags_empty(resp); + + i += part_cnt; + } + } + + /* Response: NodeEndpoints */ + rd_kafka_buf_write_arraycnt(resp, 0); + /* Response: Top-level tags */ + rd_kafka_buf_write_tags_empty(resp); + + mtx_unlock(&mcluster->lock); + + rd_kafka_mock_connection_send_response0(mconn, resp, rd_true); + + rd_kafka_topic_partition_list_destroy(requested_partitions); + return 0; + } + + /* Error response */ + rd_kafka_buf_write_i32(resp, 0); + rd_kafka_buf_write_i16(resp, err); + rd_kafka_buf_write_str(resp, rd_kafka_err2str(err), -1); + rd_kafka_buf_write_i32(resp, 0); + rd_kafka_buf_write_arraycnt(resp, 0); + rd_kafka_buf_write_arraycnt(resp, 0); + rd_kafka_buf_write_tags_empty(resp); + + rd_kafka_mock_connection_send_response0(mconn, resp, rd_true); + rd_kafka_topic_partition_list_destroy(requested_partitions); + return 0; + +err_parse: + RD_IF_FREE(requested_partitions, rd_kafka_topic_partition_list_destroy); + rd_kafka_buf_destroy(resp); + return -1; +} + /** * @brief Default request handlers */ diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index fb6d59d809..1ccd64901d 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -295,6 +295,88 @@ typedef struct rd_kafka_mock_sharegroup_member_s { rd_kafka_mock_sharegroup_t *mshgrp; /**< Share group */ } rd_kafka_mock_sharegroup_member_t; +/** + * @struct Share group member (KIP-932). + */ +typedef struct rd_kafka_mock_sgrp_member_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_member_s) link; + char *id; /**< MemberId */ + char *instance_id; /**< Group instance id */ + rd_ts_t ts_last_activity; /**< Last activity */ + int32_t current_member_epoch; /**< Current member epoch */ + int32_t target_member_epoch; /**< Target member epoch */ + rd_kafka_topic_partition_list_t *current_assignment; + rd_kafka_topic_partition_list_t *target_assignment; + rd_kafka_topic_partition_list_t *returned_assignment; + rd_list_t *subscribed_topics; + rd_list_t *subscribed_topic_names; + char *subscribed_topic_regex; + rd_bool_t left_static_membership; + struct rd_kafka_mock_connection_s *conn; + struct rd_kafka_mock_sgrp_s *sgrp; +} rd_kafka_mock_sgrp_member_t; + +/** + * @brief Share record state. + */ +enum rd_kafka_mock_sgrp_record_state_e { + RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE = 0, + RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED = 1, + RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED = 2 +}; + +typedef struct rd_kafka_mock_sgrp_record_state_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_record_state_s) link; + int64_t offset; + char *owner_member_id; + rd_ts_t lock_expiry_ts; + int32_t delivery_count; + enum rd_kafka_mock_sgrp_record_state_e state; +} rd_kafka_mock_sgrp_record_state_t; + +/** + * @brief Share partition metadata. + */ +typedef struct rd_kafka_mock_sgrp_partmeta_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_partmeta_s) link; + rd_kafka_Uuid_t topic_id; + int32_t partition; + int64_t spso; + int64_t speo; + TAILQ_HEAD(, rd_kafka_mock_sgrp_record_state_s) inflight; + int inflight_cnt; +} rd_kafka_mock_sgrp_partmeta_t; + +/** + * @brief Share fetch session. + */ +typedef struct rd_kafka_mock_sgrp_fetch_session_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_fetch_session_s) link; + char *member_id; + int32_t session_id; + int32_t session_epoch; + rd_ts_t ts_last_activity; + rd_kafka_topic_partition_list_t *partitions; +} rd_kafka_mock_sgrp_fetch_session_t; + +/** + * @brief Share group (KIP-932). + */ +typedef struct rd_kafka_mock_sgrp_s { + TAILQ_ENTRY(rd_kafka_mock_sgrp_s) link; + struct rd_kafka_mock_cluster_s *cluster; + char *id; + int32_t group_epoch; + int session_timeout_ms; + rd_kafka_timer_t session_tmr; + TAILQ_HEAD(, rd_kafka_mock_sgrp_member_s) members; + int member_cnt; + TAILQ_HEAD(, rd_kafka_mock_sgrp_partmeta_s) partitions; + int partition_cnt; + TAILQ_HEAD(, rd_kafka_mock_sgrp_fetch_session_s) fetch_sessions; + int fetch_session_cnt; +} rd_kafka_mock_sgrp_t; + /** * @struct TransactionalId + PID (+ optional sequence state) */ @@ -547,6 +629,7 @@ struct rd_kafka_mock_cluster_s { TAILQ_HEAD(, rd_kafka_mock_sgrp_s) sgrps_share; TAILQ_HEAD(, rd_kafka_mock_sharegroup_s) sharegrps; + TAILQ_HEAD(, rd_kafka_mock_sgrp_s) sgrps_share; /** Explicit coordinators (set with mock_set_coordinator()) */ TAILQ_HEAD(, rd_kafka_mock_coord_s) coords; From 52998d7627daeba5fc04be8c7574993500277510 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Thu, 5 Feb 2026 18:57:57 +0530 Subject: [PATCH 06/20] Fix SGHB return and fetcher api version --- src/rdkafka_cgrp.c | 5 +++++ src/rdkafka_fetcher.c | 5 ++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 062dd51cd7..4d9be006ff 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -6552,6 +6552,11 @@ void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg, } } + if (RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk) && + !rkcg_subscription_topics && rkcg->rkcg_subscription_topics) { + rkcg_subscription_topics = rkcg->rkcg_subscription_topics; + } + rkcg->rkcg_expedite_heartbeat_retries++; if (RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk)) { diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index 1784ac0a33..b4328452c6 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -1471,7 +1471,7 @@ void rd_kafka_ShareFetchRequest( } ApiVersion = rd_kafka_broker_ApiVersion_supported(rkb, RD_KAFKAP_ShareFetch, - 1, 1, NULL); + 0, 0, NULL); rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_ShareFetch, 1, rkbuf_size, @@ -1681,6 +1681,9 @@ void rd_kafka_ShareFetchRequest( rd_kafka_buf_write_arraycnt(rkbuf, 0); } + /* Top-level tags */ + rd_kafka_buf_write_tags_empty(rkbuf); + /* Consider Fetch requests blocking if fetch.wait.max.ms >= 1s */ if (rkb->rkb_rk->rk_conf.fetch_wait_max_ms >= 1000) rkbuf->rkbuf_flags |= RD_KAFKA_OP_F_BLOCKING; From 88699bc39c90caf9ca4aa53cb52247db40906bde Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Fri, 6 Feb 2026 18:23:51 +0530 Subject: [PATCH 07/20] Add test for share fetch API --- tests/0152-kip932-sharefetch_mockbroker.c | 475 ++++++++++++++++++++++ 1 file changed, 475 insertions(+) create mode 100644 tests/0152-kip932-sharefetch_mockbroker.c diff --git a/tests/0152-kip932-sharefetch_mockbroker.c b/tests/0152-kip932-sharefetch_mockbroker.c new file mode 100644 index 0000000000..52b29b2eac --- /dev/null +++ b/tests/0152-kip932-sharefetch_mockbroker.c @@ -0,0 +1,475 @@ +/* + * KIP-932 ShareFetch mock broker demo using the librdkafka share consumer API. + * + * This test exercises the ShareFetch path only. There is no coordinator + * or ShareAcknowledge support in the mock broker, so group management and + * ack-based state transitions are not validated here. + * + * Build (from repo root): + * cc -Isrc -o /tmp/kip932_share_consumer_mock \ + * tests/012x-kip932-sharefetch_mockbroker.c src/librdkafka.a -lz -lpthread + * + * Run: + * /tmp/kip932_share_consumer_mock + */ + +#include "rdkafka.h" +#include "rdkafka_mock.h" +#include "rdkafka_protocol.h" + +#include +#include +#include +#include + +static void die(const char *msg) { + fprintf(stderr, "error: %s\n", msg); + exit(1); +} + +static void conf_set(rd_kafka_conf_t *conf, const char *name, + const char *value) { + char errstr[512]; + if (rd_kafka_conf_set(conf, name, value, errstr, sizeof(errstr)) != + RD_KAFKA_CONF_OK) + die(errstr); +} + +typedef struct test_ctx_s { + rd_kafka_t *producer; + rd_kafka_mock_cluster_t *mcluster; + const char *bootstraps; +} test_ctx_t; + +static test_ctx_t test_ctx_new(void) { + rd_kafka_conf_t *conf = rd_kafka_conf_new(); + test_ctx_t ctx; + + memset(&ctx, 0, sizeof(ctx)); + conf_set(conf, "test.mock.num.brokers", "3"); + ctx.producer = rd_kafka_new(RD_KAFKA_PRODUCER, conf, NULL, 0); + if (!ctx.producer) + die("Failed to create producer"); + + ctx.mcluster = rd_kafka_handle_mock_cluster(ctx.producer); + if (!ctx.mcluster) + die("Failed to get mock cluster handle"); + ctx.bootstraps = rd_kafka_mock_cluster_bootstraps(ctx.mcluster); + if (!ctx.bootstraps) + die("Failed to get mock bootstraps"); + + if (rd_kafka_mock_set_apiversion( + ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to enable ShareGroupHeartbeat"); + if (rd_kafka_mock_set_apiversion( + ctx.mcluster, RD_KAFKAP_ShareFetch, 0, 0) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to enable ShareFetch"); + + return ctx; +} + +static void test_ctx_destroy(test_ctx_t *ctx) { + if (ctx->producer) + rd_kafka_destroy(ctx->producer); + memset(ctx, 0, sizeof(*ctx)); +} + +static void produce_messages(rd_kafka_t *producer, const char *topic, + int msgcnt) { + for (int i = 0; i < msgcnt; i++) { + char payload[64]; + snprintf(payload, sizeof(payload), "%s-%d", topic, i); + if (rd_kafka_producev( + producer, RD_KAFKA_V_TOPIC(topic), + RD_KAFKA_V_VALUE(payload, strlen(payload)), + RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY), + RD_KAFKA_V_END) != RD_KAFKA_RESP_ERR_NO_ERROR) { + die("Produce failed"); + } + } + rd_kafka_flush(producer, 5000); +} + +static rd_kafka_share_t *new_share_consumer(const char *bootstraps, + const char *group_id) { + rd_kafka_conf_t *conf = rd_kafka_conf_new(); + rd_kafka_share_t *consumer; + + conf_set(conf, "bootstrap.servers", bootstraps); + conf_set(conf, "group.id", group_id); + consumer = rd_kafka_share_consumer_new(conf, NULL, 0); + if (!consumer) + die("Failed to create consumer"); + rd_kafka_share_poll_set_consumer(consumer); + return consumer; +} + +static void subscribe_topics(rd_kafka_share_t *consumer, + const char **topics, int topic_cnt) { + rd_kafka_topic_partition_list_t *tpl = + rd_kafka_topic_partition_list_new(topic_cnt); + for (int i = 0; i < topic_cnt; i++) { + rd_kafka_topic_partition_list_add( + tpl, topics[i], RD_KAFKA_PARTITION_UA); + } + if (rd_kafka_share_subscribe(consumer, tpl)) + die("Subscribe failed"); + rd_kafka_topic_partition_list_destroy(tpl); +} + +static int consume_n(rd_kafka_share_t *consumer, int expected, int max_attempts) { + int consumed = 0; + int attempts = 0; + + while (consumed < expected && attempts < max_attempts) { + rd_kafka_message_t *rkmessages[100]; + size_t rcvd_msgs = 0; + rd_kafka_error_t *error; + + error = rd_kafka_share_consume_batch( + consumer, 500, rkmessages, &rcvd_msgs); + attempts++; + + if (error) { + fprintf(stderr, "consume error: %s\n", + rd_kafka_error_string(error)); + rd_kafka_error_destroy(error); + continue; + } + + for (size_t i = 0; i < rcvd_msgs; i++) { + rd_kafka_message_t *rkmsg = rkmessages[i]; + if (rkmsg->err) { + fprintf(stderr, "consume error: %s\n", + rd_kafka_message_errstr(rkmsg)); + rd_kafka_message_destroy(rkmsg); + continue; + } + printf("Consumed: %.*s\n", (int)rkmsg->len, + (const char *)rkmsg->payload); + consumed++; + rd_kafka_message_destroy(rkmsg); + } + } + + return consumed; +} + +static int run_basic_consume(void) { + const char *topic = "kip932_pos_basic"; + const int msgcnt = 5; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, msgcnt); + + consumer = new_share_consumer(ctx.bootstraps, "sg-pos-basic"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, msgcnt, 50); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == msgcnt; +} + +static int run_followup_fetch(void) { + const char *topic = "kip932_pos_followup"; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, 5); + + consumer = new_share_consumer(ctx.bootstraps, "sg-pos-followup"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, 3, 30); + consumed += consume_n(consumer, 2, 30); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == 5; +} + +static int run_multi_partition(void) { + const char *topic = "kip932_pos_multi_part"; + const int msgcnt = 6; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 2, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, msgcnt); + + consumer = new_share_consumer(ctx.bootstraps, "sg-pos-multipart"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, msgcnt, 60); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == msgcnt; +} + +static int run_multi_topic(void) { + const char *topic_a = "kip932_pos_topic_a"; + const char *topic_b = "kip932_pos_topic_b"; + const char *topics[] = {topic_a, topic_b}; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic_a, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic A"); + if (rd_kafka_mock_topic_create(ctx.mcluster, topic_b, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic B"); + + produce_messages(ctx.producer, topic_a, 2); + produce_messages(ctx.producer, topic_b, 2); + consumer = new_share_consumer(ctx.bootstraps, "sg-pos-multitopic"); + subscribe_topics(consumer, topics, 2); + consumed = consume_n(consumer, 4, 40); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == 4; +} + +static int run_empty_topic_no_records(void) { + const char *topic = "kip932_pos_empty"; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + + consumer = new_share_consumer(ctx.bootstraps, "sg-pos-empty"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, 1, 5); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == 0; +} + +static int run_negative_sharefetch_error(rd_kafka_resp_err_t err) { + const char *topic = "kip932_neg_sharefetch_error"; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, 1); + + rd_kafka_mock_push_request_errors(ctx.mcluster, RD_KAFKAP_ShareFetch, + 1, err); + + consumer = new_share_consumer(ctx.bootstraps, "sg-neg-sharefetch"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, 1, 5); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == 0; +} + +static int run_sharefetch_invalid_session_epoch(void) { + return run_negative_sharefetch_error( + RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH); +} + +static int run_sharefetch_unknown_topic_or_part(void) { + return run_negative_sharefetch_error( + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART); +} + +static int run_sghb_error(rd_kafka_resp_err_t err, int count) { + const char *topic = "kip932_neg_sghb"; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, 1); + + rd_kafka_mock_push_request_errors(ctx.mcluster, + RD_KAFKAP_ShareGroupHeartbeat, count, + err); + + consumer = new_share_consumer(ctx.bootstraps, "sg-neg-sghb"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, 1, 5); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == 0; +} + +static int run_sghb_coord_unavailable(void) { + return run_sghb_error( + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, 50); +} + +static int run_topic_error(rd_kafka_resp_err_t err) { + const char *topic = "kip932_neg_topic_error"; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, 1); + rd_kafka_mock_topic_set_error(ctx.mcluster, topic, err); + + consumer = new_share_consumer(ctx.bootstraps, "sg-neg-topicerr"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, 1, 5); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == 0; +} + +static int run_topic_error_unknown_topic_or_part(void) { + return run_topic_error( + RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART); +} + +static int run_unknown_topic_subscription(void) { + const char *topic = "kip932_neg_unknown_topic"; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + consumer = new_share_consumer(ctx.bootstraps, "sg-neg-unknown-topic"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, 1, 5); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == 0; +} + +static int run_empty_fetch_no_records(void) { + const char *topic = "kip932_neg_empty_fetch"; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + + consumer = new_share_consumer(ctx.bootstraps, "sg-neg-empty"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, 1, 5); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == 0; +} + +static int run_sharefetch_session_expiry_rtt(void) { + const char *topic = "kip932_rtt_expiry"; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + rd_kafka_mock_set_group_consumer_session_timeout_ms(ctx.mcluster, 200); + rd_kafka_mock_broker_set_rtt(ctx.mcluster, 1, 500); + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, 2); + + consumer = new_share_consumer(ctx.bootstraps, "sg-rtt-expiry"); + subscribe_topics(consumer, &topic, 1); + + consumed = consume_n(consumer, 1, 20); + usleep(1000 * 1000); /* allow session timeout to elapse */ + consumed += consume_n(consumer, 1, 20); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == 2; +} + +static int run_test(const char *name, int (*fn)(void)) { + int ok = fn(); + printf("[%s] %s\n", ok ? "OK" : "FAIL", name); + return ok ? 0 : 1; +} + +int main(void) { + int failures = 0; + + printf("ShareFetch test scenarios\n"); + + /* Positive scenarios */ + failures += run_test("basic_consume", run_basic_consume); + failures += run_test("followup_fetch", run_followup_fetch); + failures += run_test("multi_partition", run_multi_partition); + failures += run_test("multi_topic", run_multi_topic); + failures += run_test("empty_topic_no_records", run_empty_topic_no_records); + failures += run_test("sharefetch_session_expiry_rtt", + run_sharefetch_session_expiry_rtt); + + /* Negative scenarios */ + failures += run_test("sharefetch_invalid_session_epoch", + run_sharefetch_invalid_session_epoch); + failures += run_test("sharefetch_unknown_topic_or_part", + run_sharefetch_unknown_topic_or_part); + failures += run_test("sghb_coord_unavailable", + run_sghb_coord_unavailable); + failures += run_test("topic_error_unknown_topic_or_part", + run_topic_error_unknown_topic_or_part); + failures += run_test("unknown_topic_subscription", + run_unknown_topic_subscription); + failures += run_test("empty_fetch_no_records", + run_empty_fetch_no_records); + + printf("Failures: %d\n", failures); + return failures ? 1 : 0; +} + From 0723bc9454ae1811c7de6f542037e722294ac0b3 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Mon, 9 Feb 2026 15:17:37 +0530 Subject: [PATCH 08/20] Fix session workflow --- src/rdkafka_mock_cgrp.c | 2 +- src/rdkafka_mock_handlers.c | 106 +++++++++++++++++++++++------------- src/rdkafka_mock_int.h | 3 +- 3 files changed, 71 insertions(+), 40 deletions(-) diff --git a/src/rdkafka_mock_cgrp.c b/src/rdkafka_mock_cgrp.c index c0154cd873..83cce18a2b 100644 --- a/src/rdkafka_mock_cgrp.c +++ b/src/rdkafka_mock_cgrp.c @@ -1879,7 +1879,7 @@ rd_kafka_mock_sgrp_partmeta_destroy(rd_kafka_mock_sgrp_partmeta_t *pmeta) { /** * @brief Destroy share fetch session. */ -static void rd_kafka_mock_sgrp_fetch_session_destroy( +void rd_kafka_mock_sgrp_fetch_session_destroy( rd_kafka_mock_sgrp_fetch_session_t *session) { rd_free(session->member_id); RD_IF_FREE(session->partitions, rd_kafka_topic_partition_list_destroy); diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 5efce2147a..020eeb97e5 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -4028,20 +4028,20 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_mock_cluster_t *mcluster = mconn->broker->cluster; rd_kafka_buf_t *resp = rd_kafka_mock_buf_new_response(rkbuf); rd_kafkap_str_t GroupId, MemberId; - int32_t SessionId = -1, SessionEpoch = -1, MaxWaitMs = 0, MinBytes = 0, + int32_t SessionEpoch = -1, MaxWaitMs = 0, MinBytes = 0, MaxBytes = 0, MaxRecords = 0, BatchSize = 0; int32_t TopicsCnt; rd_kafka_topic_partition_list_t *requested_partitions = NULL; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; rd_kafka_mock_sgrp_t *sgrp = NULL; rd_kafka_mock_sgrp_fetch_session_t *session = NULL; - static int32_t next_session_id = 1; (void)log_decode_errors; rd_kafka_buf_read_str(rkbuf, &GroupId); rd_kafka_buf_read_str(rkbuf, &MemberId); - /* Fetcher sends SessionEpoch only (no SessionId) */ + /* KIP-932: ShareFetch has ShareSessionEpoch only, no SessionId. + * Sessions are keyed by (GroupId, MemberId). */ rd_kafka_buf_read_i32(rkbuf, &SessionEpoch); rd_kafka_buf_read_i32(rkbuf, &MaxWaitMs); rd_kafka_buf_read_i32(rkbuf, &MinBytes); @@ -4095,12 +4095,12 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK", "ShareFetch parsed: group %.*s member %.*s " - "session_id %" PRId32 " session_epoch %" PRId32 + "session_epoch %" PRId32 " max_wait %" PRId32 " min_bytes %" PRId32 " max_bytes %" PRId32 " max_records %" PRId32 " batch_size %" PRId32, RD_KAFKAP_STR_PR(&GroupId), RD_KAFKAP_STR_PR(&MemberId), - SessionId, SessionEpoch, MaxWaitMs, MinBytes, MaxBytes, + SessionEpoch, MaxWaitMs, MinBytes, MaxBytes, MaxRecords, BatchSize); err = rd_kafka_mock_next_request_error(mconn, resp); @@ -4114,49 +4114,39 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, int64_t acquired_bytes = 0; rd_ts_t now = rd_clock(); - // Handle ShareFetch session management + /* KIP-932 session management. + * Sessions are keyed by (GroupId, MemberId). + * SessionEpoch: 0 = open new session, + * -1 = close session, + * >0 = continue (must match expected epoch). */ mtx_lock(&mcluster->lock); sgrp = rd_kafka_mock_sgrp_get(mcluster, &GroupId); - if (SessionId < 0) { - TAILQ_FOREACH(session, &sgrp->fetch_sessions, link) { - if (!rd_kafkap_str_cmp_str(&MemberId, - session->member_id)) - break; - } + /* Look up existing session by MemberId */ + TAILQ_FOREACH(session, &sgrp->fetch_sessions, link) { + if (!rd_kafkap_str_cmp_str(&MemberId, + session->member_id)) + break; + } + + if (SessionEpoch == 0) { + /* Open a new session (or reuse if one already exists + * for this member). */ if (!session) { session = rd_calloc(1, sizeof(*session)); - session->member_id = RD_KAFKAP_STR_DUP(&MemberId); - session->session_id = next_session_id++; + session->member_id = + RD_KAFKAP_STR_DUP(&MemberId); session->session_epoch = 0; - session->ts_last_activity = rd_clock(); session->partitions = rd_kafka_topic_partition_list_copy( requested_partitions); - TAILQ_INSERT_TAIL(&sgrp->fetch_sessions, session, - link); + TAILQ_INSERT_TAIL(&sgrp->fetch_sessions, + session, link); sgrp->fetch_session_cnt++; - } - } else { - TAILQ_FOREACH(session, &sgrp->fetch_sessions, link) { - if (session->session_id != SessionId) - continue; - if (rd_kafkap_str_cmp_str(&MemberId, - session->member_id)) - continue; - break; - } - } - - if (!session) { - err = RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH; - } else if (SessionEpoch != -1 && - SessionEpoch != session->session_epoch) { - err = RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH; - } else { - if (!rd_kafka_mock_tplist_equal_by_id( - requested_partitions, session->partitions)) { - session->session_epoch++; + } else { + /* Session already exists for this member; + * reset it. */ + session->session_epoch = 0; RD_IF_FREE( session->partitions, rd_kafka_topic_partition_list_destroy); @@ -4164,7 +4154,47 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_topic_partition_list_copy( requested_partitions); } + } else if (SessionEpoch == -1) { + /* Close the session. */ + if (session) { + TAILQ_REMOVE(&sgrp->fetch_sessions, session, + link); + sgrp->fetch_session_cnt--; + rd_kafka_mock_sgrp_fetch_session_destroy( + session); + session = NULL; + } + /* Closing a non-existent session is not an error; + * proceed with an empty response. */ + } else { + /* Continue existing session: validate epoch. */ + if (!session) { + err = + RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH; + } else if (SessionEpoch != + session->session_epoch) { + err = + RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH; + } else { + /* Update partition list if changed */ + if (!rd_kafka_mock_tplist_equal_by_id( + requested_partitions, + session->partitions)) { + RD_IF_FREE( + session->partitions, + rd_kafka_topic_partition_list_destroy); + session->partitions = + rd_kafka_topic_partition_list_copy( + requested_partitions); + } + } + } + + /* For all successful, non-close requests: update activity + * timestamp and increment epoch for next request. */ + if (!err && session) { session->ts_last_activity = rd_clock(); + session->session_epoch++; } if (!err && sgrp) { diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index 1ccd64901d..8159551cc2 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -353,7 +353,6 @@ typedef struct rd_kafka_mock_sgrp_partmeta_s { typedef struct rd_kafka_mock_sgrp_fetch_session_s { TAILQ_ENTRY(rd_kafka_mock_sgrp_fetch_session_s) link; char *member_id; - int32_t session_id; int32_t session_epoch; rd_ts_t ts_last_activity; rd_kafka_topic_partition_list_t *partitions; @@ -754,6 +753,8 @@ rd_kafka_mock_sgrp_t * rd_kafka_mock_sgrp_get(rd_kafka_mock_cluster_t *mcluster, const rd_kafkap_str_t *GroupId); void rd_kafka_mock_sgrp_destroy(rd_kafka_mock_sgrp_t *sgrp); +void rd_kafka_mock_sgrp_fetch_session_destroy( + rd_kafka_mock_sgrp_fetch_session_t *session); rd_kafka_mock_broker_t * rd_kafka_mock_cluster_get_coord(rd_kafka_mock_cluster_t *mcluster, From 80a42f143bb3a1469261bd29e89cb0ccd3af58d7 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Mon, 9 Feb 2026 15:35:49 +0530 Subject: [PATCH 09/20] Update API version --- src/rdkafka_fetcher.c | 2 +- src/rdkafka_mock_handlers.c | 2 +- tests/0152-kip932-sharefetch_mockbroker.c | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index b4328452c6..3628090330 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -1471,7 +1471,7 @@ void rd_kafka_ShareFetchRequest( } ApiVersion = rd_kafka_broker_ApiVersion_supported(rkb, RD_KAFKAP_ShareFetch, - 0, 0, NULL); + 1, 1, NULL); rkbuf = rd_kafka_buf_new_flexver_request(rkb, RD_KAFKAP_ShareFetch, 1, rkbuf_size, diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 020eeb97e5..7d5940e139 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -4441,7 +4441,7 @@ const struct rd_kafka_mock_api_handler {0, 0, 0, rd_kafka_mock_handle_GetTelemetrySubscriptions}, [RD_KAFKAP_PushTelemetry] = {0, 0, 0, rd_kafka_mock_handle_PushTelemetry}, - [RD_KAFKAP_ShareFetch] = {0, 0, 0, rd_kafka_mock_handle_ShareFetch}, + [RD_KAFKAP_ShareFetch] = {1, 1, 1, rd_kafka_mock_handle_ShareFetch}, }; diff --git a/tests/0152-kip932-sharefetch_mockbroker.c b/tests/0152-kip932-sharefetch_mockbroker.c index 52b29b2eac..1dd4e9994c 100644 --- a/tests/0152-kip932-sharefetch_mockbroker.c +++ b/tests/0152-kip932-sharefetch_mockbroker.c @@ -63,7 +63,7 @@ static test_ctx_t test_ctx_new(void) { RD_KAFKA_RESP_ERR_NO_ERROR) die("Failed to enable ShareGroupHeartbeat"); if (rd_kafka_mock_set_apiversion( - ctx.mcluster, RD_KAFKAP_ShareFetch, 0, 0) != + ctx.mcluster, RD_KAFKAP_ShareFetch, 1, 1) != RD_KAFKA_RESP_ERR_NO_ERROR) die("Failed to enable ShareFetch"); From fce9a47733ba7c8822d55c7ddbc2c6efa1067da3 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Mon, 9 Feb 2026 20:34:28 +0530 Subject: [PATCH 10/20] Add handing for ForgottenTopicsData --- src/rdkafka_mock_handlers.c | 105 ++++++++++++++++++++++ tests/0152-kip932-sharefetch_mockbroker.c | 48 ++++++++++ 2 files changed, 153 insertions(+) diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 7d5940e139..3e8dfb5538 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -4032,6 +4032,7 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, MaxBytes = 0, MaxRecords = 0, BatchSize = 0; int32_t TopicsCnt; rd_kafka_topic_partition_list_t *requested_partitions = NULL; + rd_kafka_topic_partition_list_t *forgotten_partitions = NULL; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; rd_kafka_mock_sgrp_t *sgrp = NULL; rd_kafka_mock_sgrp_fetch_session_t *session = NULL; @@ -4091,6 +4092,51 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_skip_tags(rkbuf); } + /* ForgottenTopicsData */ + { + int32_t ForgottenTopicsCnt; + rd_kafka_buf_read_arraycnt(rkbuf, &ForgottenTopicsCnt, + RD_KAFKAP_TOPICS_MAX); + if (ForgottenTopicsCnt > 0) + forgotten_partitions = + rd_kafka_topic_partition_list_new( + ForgottenTopicsCnt); + while (ForgottenTopicsCnt-- > 0) { + rd_kafka_Uuid_t ForgTopicId = RD_KAFKA_UUID_ZERO; + int32_t ForgPartCnt; + rd_kafka_buf_read_uuid(rkbuf, &ForgTopicId); + rd_kafka_buf_read_arraycnt(rkbuf, &ForgPartCnt, + RD_KAFKAP_PARTITIONS_MAX); + while (ForgPartCnt-- > 0) { + int32_t ForgPartition; + rd_kafka_topic_partition_t *ftp; + rd_kafka_buf_read_i32(rkbuf, &ForgPartition); + + /* Record in forgotten list for session + * removal and inflight release. */ + ftp = rd_kafka_topic_partition_list_add( + forgotten_partitions, "", ForgPartition); + rd_kafka_topic_partition_set_topic_id( + ftp, ForgTopicId); + + /* Remove from requested_partitions so they + * are not fetched in this request. */ + if (requested_partitions) { + int idx = + rd_kafka_topic_partition_list_find_idx_by_id( + requested_partitions, + ForgTopicId, ForgPartition); + if (idx >= 0) + rd_kafka_topic_partition_list_del_by_idx( + requested_partitions, idx); + } + } + /* ForgottenTopic tags */ + rd_kafka_buf_skip_tags(rkbuf); + } + } + + /* Top-level tags */ rd_kafka_buf_skip_tags(rkbuf); rd_kafka_dbg(mconn->broker->cluster->rk, MOCK, "MOCK", @@ -4197,6 +4243,59 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, session->session_epoch++; } + /* Remove forgotten partitions from session and release + * any in-flight ACQUIRED records owned by this member. */ + if (!err && session && forgotten_partitions && + forgotten_partitions->cnt > 0) { + rd_kafka_topic_partition_t *ftp; + RD_KAFKA_TPLIST_FOREACH(ftp, forgotten_partitions) { + rd_kafka_Uuid_t ftid = + rd_kafka_topic_partition_get_topic_id(ftp); + rd_kafka_mock_sgrp_partmeta_t *pmeta; + + /* Remove from session partition list */ + if (session->partitions) { + int idx = + rd_kafka_topic_partition_list_find_idx_by_id( + session->partitions, ftid, + ftp->partition); + if (idx >= 0) + rd_kafka_topic_partition_list_del_by_idx( + session->partitions, idx); + } + + /* Release ACQUIRED records for this + * member on this partition. */ + pmeta = rd_kafka_mock_sgrp_partmeta_find( + sgrp, ftid, ftp->partition); + if (pmeta) { + rd_kafka_mock_sgrp_record_state_t + *state, *tmp; + TAILQ_FOREACH_SAFE( + state, &pmeta->inflight, link, + tmp) { + if (state->state != + RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) + continue; + if (!state->owner_member_id) + continue; + if (rd_kafkap_str_cmp_str( + &MemberId, + state + ->owner_member_id)) + continue; + /* Release: mark AVAILABLE */ + state->state = + RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE; + rd_free( + state->owner_member_id); + state->owner_member_id = NULL; + state->lock_expiry_ts = 0; + } + } + } + } + if (!err && sgrp) { rd_kafka_topic_partition_t *rktpar; RD_KAFKA_TPLIST_FOREACH(rktpar, requested_partitions) { @@ -4381,6 +4480,8 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_mock_connection_send_response0(mconn, resp, rd_true); rd_kafka_topic_partition_list_destroy(requested_partitions); + RD_IF_FREE(forgotten_partitions, + rd_kafka_topic_partition_list_destroy); return 0; } @@ -4395,10 +4496,14 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_mock_connection_send_response0(mconn, resp, rd_true); rd_kafka_topic_partition_list_destroy(requested_partitions); + RD_IF_FREE(forgotten_partitions, + rd_kafka_topic_partition_list_destroy); return 0; err_parse: RD_IF_FREE(requested_partitions, rd_kafka_topic_partition_list_destroy); + RD_IF_FREE(forgotten_partitions, + rd_kafka_topic_partition_list_destroy); rd_kafka_buf_destroy(resp); return -1; } diff --git a/tests/0152-kip932-sharefetch_mockbroker.c b/tests/0152-kip932-sharefetch_mockbroker.c index 1dd4e9994c..a5fbcf18b9 100644 --- a/tests/0152-kip932-sharefetch_mockbroker.c +++ b/tests/0152-kip932-sharefetch_mockbroker.c @@ -435,6 +435,53 @@ static int run_sharefetch_session_expiry_rtt(void) { return consumed == 2; } +static int run_forgotten_topics(void) { + const char *topic_a = "kip932_forgotten_a"; + const char *topic_b = "kip932_forgotten_b"; + const char *both[] = {topic_a, topic_b}; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic_a, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic A"); + if (rd_kafka_mock_topic_create(ctx.mcluster, topic_b, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic B"); + + /* Produce 2 messages to each topic */ + produce_messages(ctx.producer, topic_a, 2); + produce_messages(ctx.producer, topic_b, 2); + + /* Subscribe to both topics and consume all 4 messages */ + consumer = new_share_consumer(ctx.bootstraps, "sg-forgotten"); + subscribe_topics(consumer, both, 2); + consumed = consume_n(consumer, 4, 40); + printf(" forgotten_topics: consumed %d/4 from both topics\n", + consumed); + + /* Re-subscribe to only topic_a (topic_b becomes forgotten) */ + subscribe_topics(consumer, &topic_a, 1); + + /* Produce 2 more messages to topic_a */ + produce_messages(ctx.producer, topic_a, 2); + + /* Consume the 2 new messages — only topic_a should deliver */ + consumed += consume_n(consumer, 2, 30); + printf(" forgotten_topics: consumed %d/6 total after forget\n", + consumed); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + /* We expect at least the 4 initial + 2 from topic_a = 6. + * Depending on timing the consumer may or may not have already + * received all messages from the first round, so we accept >= 4. */ + return consumed >= 4; +} + static int run_test(const char *name, int (*fn)(void)) { int ok = fn(); printf("[%s] %s\n", ok ? "OK" : "FAIL", name); @@ -454,6 +501,7 @@ int main(void) { failures += run_test("empty_topic_no_records", run_empty_topic_no_records); failures += run_test("sharefetch_session_expiry_rtt", run_sharefetch_session_expiry_rtt); + failures += run_test("forgotten_topics", run_forgotten_topics); /* Negative scenarios */ failures += run_test("sharefetch_invalid_session_epoch", From 1045dc897792a39831c7e5e9ded59b85773e26fd Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Mon, 9 Feb 2026 20:54:09 +0530 Subject: [PATCH 11/20] Add lock expiry logic --- src/rdkafka_mock_cgrp.c | 73 ++++++++++++++++++++++- src/rdkafka_mock_handlers.c | 5 +- src/rdkafka_mock_int.h | 2 + tests/0152-kip932-sharefetch_mockbroker.c | 54 +++++++++++++++++ 4 files changed, 132 insertions(+), 2 deletions(-) diff --git a/src/rdkafka_mock_cgrp.c b/src/rdkafka_mock_cgrp.c index 83cce18a2b..ffaae173c2 100644 --- a/src/rdkafka_mock_cgrp.c +++ b/src/rdkafka_mock_cgrp.c @@ -1887,7 +1887,67 @@ void rd_kafka_mock_sgrp_fetch_session_destroy( } /** - * @brief Check if any share fetch sessions have expired. + * @brief Release all ACQUIRED records owned by \p member_id across all + * share-partition metadata in the share group. + * + * This is called when a session is closed (epoch=-1), when a session + * times out, or as part of periodic lock-expiry reclamation. + */ +void rd_kafka_mock_sgrp_release_member_locks(rd_kafka_mock_sgrp_t *sgrp, + const char *member_id) { + rd_kafka_mock_sgrp_partmeta_t *pmeta; + + TAILQ_FOREACH(pmeta, &sgrp->partitions, link) { + rd_kafka_mock_sgrp_record_state_t *state, *tmp; + TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { + if (state->state != + RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) + continue; + if (!state->owner_member_id) + continue; + if (strcmp(state->owner_member_id, member_id) != 0) + continue; + + state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE; + rd_free(state->owner_member_id); + state->owner_member_id = NULL; + state->lock_expiry_ts = 0; + } + } +} + +/** + * @brief Proactively release any expired acquisition locks. + * + * Iterates all partition metadata in the share group and flips + * ACQUIRED records whose lock_expiry_ts has passed back to AVAILABLE. + */ +static void +rd_kafka_mock_sgrp_expire_locks(rd_kafka_mock_sgrp_t *sgrp, rd_ts_t now) { + rd_kafka_mock_sgrp_partmeta_t *pmeta; + + TAILQ_FOREACH(pmeta, &sgrp->partitions, link) { + rd_kafka_mock_sgrp_record_state_t *state, *tmp; + TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { + if (state->state != + RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) + continue; + if (!state->lock_expiry_ts || + state->lock_expiry_ts > now) + continue; + + /* Lock has expired — release back to AVAILABLE */ + state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE; + RD_IF_FREE(state->owner_member_id, rd_free); + state->owner_member_id = NULL; + state->lock_expiry_ts = 0; + } + } +} + +/** + * @brief Periodic timer: expire stale share-fetch sessions and + * proactively reclaim expired acquisition locks. */ static void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts, void *arg) { @@ -1897,16 +1957,27 @@ static void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts, (void)rkts; + /* 1. Expire stale sessions and release their member locks. */ TAILQ_FOREACH_SAFE(session, &sgrp->fetch_sessions, link, tmp) { if (session->ts_last_activity + (sgrp->session_timeout_ms * 1000) > now) continue; + /* Release all locks held by this member before + * destroying the session. */ + rd_kafka_mock_sgrp_release_member_locks(sgrp, + session->member_id); + TAILQ_REMOVE(&sgrp->fetch_sessions, session, link); sgrp->fetch_session_cnt--; rd_kafka_mock_sgrp_fetch_session_destroy(session); } + + /* 2. Proactively reclaim any expired acquisition locks. + * This catches records whose owning consumer crashed + * without closing its session cleanly. */ + rd_kafka_mock_sgrp_expire_locks(sgrp, now); } /** diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 3e8dfb5538..954c0410a2 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -4201,8 +4201,11 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, requested_partitions); } } else if (SessionEpoch == -1) { - /* Close the session. */ + /* Close the session and release all locks held + * by this member. */ if (session) { + rd_kafka_mock_sgrp_release_member_locks( + sgrp, session->member_id); TAILQ_REMOVE(&sgrp->fetch_sessions, session, link); sgrp->fetch_session_cnt--; diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index 8159551cc2..8db071541e 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -755,6 +755,8 @@ rd_kafka_mock_sgrp_get(rd_kafka_mock_cluster_t *mcluster, void rd_kafka_mock_sgrp_destroy(rd_kafka_mock_sgrp_t *sgrp); void rd_kafka_mock_sgrp_fetch_session_destroy( rd_kafka_mock_sgrp_fetch_session_t *session); +void rd_kafka_mock_sgrp_release_member_locks(rd_kafka_mock_sgrp_t *sgrp, + const char *member_id); rd_kafka_mock_broker_t * rd_kafka_mock_cluster_get_coord(rd_kafka_mock_cluster_t *mcluster, diff --git a/tests/0152-kip932-sharefetch_mockbroker.c b/tests/0152-kip932-sharefetch_mockbroker.c index a5fbcf18b9..08662f718b 100644 --- a/tests/0152-kip932-sharefetch_mockbroker.c +++ b/tests/0152-kip932-sharefetch_mockbroker.c @@ -482,6 +482,58 @@ static int run_forgotten_topics(void) { return consumed >= 4; } +/** + * @brief Multi-consumer lock expiry test. + * + * Consumer A acquires records, then crashes (destroyed without close). + * After the lock expiry timeout, consumer B should be able to pick up + * the same records because the proactive lock-expiry scan releases them. + */ +static int run_multi_consumer_lock_expiry(void) { + const char *topic = "kip932_multi_consumer_lock"; + const int msgcnt = 3; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer_a, *consumer_b; + int consumed_a, consumed_b; + + /* Use a short session/lock timeout so the test runs quickly. */ + rd_kafka_mock_set_group_consumer_session_timeout_ms(ctx.mcluster, 500); + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, msgcnt); + + /* Consumer A: subscribe and consume all records (acquires locks). */ + consumer_a = + new_share_consumer(ctx.bootstraps, "sg-multi-consumer-lock"); + subscribe_topics(consumer_a, &topic, 1); + consumed_a = consume_n(consumer_a, msgcnt, 50); + printf(" multi_consumer: A consumed %d/%d\n", consumed_a, msgcnt); + + /* Simulate crash: destroy consumer A without calling close. + * The session will time out and the proactive lock-expiry + * timer will release A's locks. */ + rd_kafka_share_destroy(consumer_a); + + /* Wait for locks to expire (session_timeout=500ms, add margin). */ + usleep(1500 * 1000); + + /* Consumer B: joins the same share group, should get the same + * records once the locks have been released. */ + consumer_b = + new_share_consumer(ctx.bootstraps, "sg-multi-consumer-lock"); + subscribe_topics(consumer_b, &topic, 1); + consumed_b = consume_n(consumer_b, msgcnt, 50); + printf(" multi_consumer: B consumed %d/%d\n", consumed_b, msgcnt); + + rd_kafka_share_consumer_close(consumer_b); + rd_kafka_share_destroy(consumer_b); + test_ctx_destroy(&ctx); + + return consumed_a == msgcnt && consumed_b == msgcnt; +} + static int run_test(const char *name, int (*fn)(void)) { int ok = fn(); printf("[%s] %s\n", ok ? "OK" : "FAIL", name); @@ -502,6 +554,8 @@ int main(void) { failures += run_test("sharefetch_session_expiry_rtt", run_sharefetch_session_expiry_rtt); failures += run_test("forgotten_topics", run_forgotten_topics); + failures += run_test("multi_consumer_lock_expiry", + run_multi_consumer_lock_expiry); /* Negative scenarios */ failures += run_test("sharefetch_invalid_session_epoch", From 4121f894cc3cb2aa485f62e41f998cacf29b4eda Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Mon, 9 Feb 2026 21:37:05 +0530 Subject: [PATCH 12/20] Remove duplicates with sghb --- src/rdkafka_mock.c | 5 - src/rdkafka_mock_cgrp.c | 125 +++------------------- src/rdkafka_mock_handlers.c | 8 +- src/rdkafka_mock_int.h | 62 +++-------- src/rdkafka_mock_sharegrp.c | 34 ++++++ tests/0152-kip932-sharefetch_mockbroker.c | 24 ++++- 6 files changed, 87 insertions(+), 171 deletions(-) diff --git a/src/rdkafka_mock.c b/src/rdkafka_mock.c index 0bd13539a8..3c8b00988f 100644 --- a/src/rdkafka_mock.c +++ b/src/rdkafka_mock.c @@ -2778,7 +2778,6 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) { rd_kafka_mock_cgrp_consumer_t *mcgrp_consumer; rd_kafka_mock_sgrp_t *msgrp; rd_kafka_mock_sharegroup_t *mshgrp; - rd_kafka_mock_sgrp_t *msgrp; rd_kafka_mock_coord_t *mcoord; rd_kafka_mock_error_stack_t *errstack; thrd_t dummy_rkb_thread; @@ -2803,9 +2802,6 @@ static void rd_kafka_mock_cluster_destroy0(rd_kafka_mock_cluster_t *mcluster) { while ((mshgrp = TAILQ_FIRST(&mcluster->sharegrps))) rd_kafka_mock_sharegroup_destroy(mshgrp); - while ((msgrp = TAILQ_FIRST(&mcluster->sgrps_share))) - rd_kafka_mock_sgrp_destroy(msgrp); - while ((mcoord = TAILQ_FIRST(&mcluster->coords))) rd_kafka_mock_coord_destroy(mcluster, mcoord); @@ -2926,7 +2922,6 @@ rd_kafka_mock_cluster_t *rd_kafka_mock_cluster_new(rd_kafka_t *rk, TAILQ_INIT(&mcluster->sgrps_share); rd_kafka_mock_sharegrps_init(mcluster); - TAILQ_INIT(&mcluster->sgrps_share); TAILQ_INIT(&mcluster->coords); diff --git a/src/rdkafka_mock_cgrp.c b/src/rdkafka_mock_cgrp.c index ffaae173c2..23177d2a0e 100644 --- a/src/rdkafka_mock_cgrp.c +++ b/src/rdkafka_mock_cgrp.c @@ -1893,11 +1893,12 @@ void rd_kafka_mock_sgrp_fetch_session_destroy( * This is called when a session is closed (epoch=-1), when a session * times out, or as part of periodic lock-expiry reclamation. */ -void rd_kafka_mock_sgrp_release_member_locks(rd_kafka_mock_sgrp_t *sgrp, - const char *member_id) { +void rd_kafka_mock_sgrp_release_member_locks( + rd_kafka_mock_sharegroup_t *mshgrp, + const char *member_id) { rd_kafka_mock_sgrp_partmeta_t *pmeta; - TAILQ_FOREACH(pmeta, &sgrp->partitions, link) { + TAILQ_FOREACH(pmeta, &mshgrp->partitions, link) { rd_kafka_mock_sgrp_record_state_t *state, *tmp; TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { if (state->state != @@ -1923,10 +1924,11 @@ void rd_kafka_mock_sgrp_release_member_locks(rd_kafka_mock_sgrp_t *sgrp, * ACQUIRED records whose lock_expiry_ts has passed back to AVAILABLE. */ static void -rd_kafka_mock_sgrp_expire_locks(rd_kafka_mock_sgrp_t *sgrp, rd_ts_t now) { +rd_kafka_mock_sgrp_expire_locks(rd_kafka_mock_sharegroup_t *mshgrp, + rd_ts_t now) { rd_kafka_mock_sgrp_partmeta_t *pmeta; - TAILQ_FOREACH(pmeta, &sgrp->partitions, link) { + TAILQ_FOREACH(pmeta, &mshgrp->partitions, link) { rd_kafka_mock_sgrp_record_state_t *state, *tmp; TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { if (state->state != @@ -1949,130 +1951,35 @@ rd_kafka_mock_sgrp_expire_locks(rd_kafka_mock_sgrp_t *sgrp, rd_ts_t now) { * @brief Periodic timer: expire stale share-fetch sessions and * proactively reclaim expired acquisition locks. */ -static void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts, - void *arg) { - rd_kafka_mock_sgrp_t *sgrp = arg; +void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts, + void *arg) { + rd_kafka_mock_sharegroup_t *mshgrp = arg; rd_kafka_mock_sgrp_fetch_session_t *session, *tmp; rd_ts_t now = rd_clock(); (void)rkts; /* 1. Expire stale sessions and release their member locks. */ - TAILQ_FOREACH_SAFE(session, &sgrp->fetch_sessions, link, tmp) { + TAILQ_FOREACH_SAFE(session, &mshgrp->fetch_sessions, link, tmp) { if (session->ts_last_activity + - (sgrp->session_timeout_ms * 1000) > + (mshgrp->session_timeout_ms * 1000) > now) continue; /* Release all locks held by this member before * destroying the session. */ - rd_kafka_mock_sgrp_release_member_locks(sgrp, + rd_kafka_mock_sgrp_release_member_locks(mshgrp, session->member_id); - TAILQ_REMOVE(&sgrp->fetch_sessions, session, link); - sgrp->fetch_session_cnt--; + TAILQ_REMOVE(&mshgrp->fetch_sessions, session, link); + mshgrp->fetch_session_cnt--; rd_kafka_mock_sgrp_fetch_session_destroy(session); } /* 2. Proactively reclaim any expired acquisition locks. * This catches records whose owning consumer crashed * without closing its session cleanly. */ - rd_kafka_mock_sgrp_expire_locks(sgrp, now); -} - -/** - * @brief Find share group in cluster by GroupId. - * - * @locks mcluster->lock MUST be held. - */ -rd_kafka_mock_sgrp_t * -rd_kafka_mock_sgrp_find(rd_kafka_mock_cluster_t *mcluster, - const rd_kafkap_str_t *GroupId) { - rd_kafka_mock_sgrp_t *sgrp; - TAILQ_FOREACH(sgrp, &mcluster->sgrps_share, link) { - if (!rd_kafkap_str_cmp_str(GroupId, sgrp->id)) - return sgrp; - } - - return NULL; -} - -/** - * @brief Find or create a share group for ShareFetch. - * - * @locks mcluster->lock MUST be held. - */ -rd_kafka_mock_sgrp_t * -rd_kafka_mock_sgrp_get(rd_kafka_mock_cluster_t *mcluster, - const rd_kafkap_str_t *GroupId) { - rd_kafka_mock_sgrp_t *sgrp; - - sgrp = rd_kafka_mock_sgrp_find(mcluster, GroupId); - if (sgrp) - return sgrp; - - sgrp = rd_calloc(1, sizeof(*sgrp)); - sgrp->cluster = mcluster; - sgrp->id = RD_KAFKAP_STR_DUP(GroupId); - sgrp->group_epoch = 1; - sgrp->session_timeout_ms = - mcluster->defaults.group_consumer_session_timeout_ms; - TAILQ_INIT(&sgrp->members); - TAILQ_INIT(&sgrp->partitions); - TAILQ_INIT(&sgrp->fetch_sessions); - rd_kafka_timer_start(&mcluster->timers, &sgrp->session_tmr, - 1000 * 1000 /*1s*/, - rd_kafka_mock_sgrp_fetch_session_tmr_cb, sgrp); - - TAILQ_INSERT_TAIL(&mcluster->sgrps_share, sgrp, link); - - return sgrp; -} - -/** - * @brief Destroy share group and all of its ShareFetch state. - * - * @locks mcluster->lock MUST be held. - */ -void rd_kafka_mock_sgrp_destroy(rd_kafka_mock_sgrp_t *sgrp) { - rd_kafka_mock_sgrp_member_t *member; - rd_kafka_mock_sgrp_partmeta_t *pmeta; - rd_kafka_mock_sgrp_fetch_session_t *session; - - TAILQ_REMOVE(&sgrp->cluster->sgrps_share, sgrp, link); - - rd_kafka_timer_stop(&sgrp->cluster->timers, &sgrp->session_tmr, rd_true); - - while ((member = TAILQ_FIRST(&sgrp->members))) { - TAILQ_REMOVE(&sgrp->members, member, link); - sgrp->member_cnt--; - rd_free(member->id); - RD_IF_FREE(member->instance_id, rd_free); - RD_IF_FREE(member->target_assignment, - rd_kafka_topic_partition_list_destroy); - RD_IF_FREE(member->current_assignment, - rd_kafka_topic_partition_list_destroy); - RD_IF_FREE(member->returned_assignment, - rd_kafka_topic_partition_list_destroy); - RD_IF_FREE(member->subscribed_topics, rd_list_destroy_free); - RD_IF_FREE(member->subscribed_topic_names, rd_list_destroy_free); - RD_IF_FREE(member->subscribed_topic_regex, rd_free); - rd_free(member); - } - - while ((pmeta = TAILQ_FIRST(&sgrp->partitions))) { - TAILQ_REMOVE(&sgrp->partitions, pmeta, link); - rd_kafka_mock_sgrp_partmeta_destroy(pmeta); - } - - while ((session = TAILQ_FIRST(&sgrp->fetch_sessions))) { - TAILQ_REMOVE(&sgrp->fetch_sessions, session, link); - sgrp->fetch_session_cnt--; - rd_kafka_mock_sgrp_fetch_session_destroy(session); - } - - rd_free(sgrp->id); - rd_free(sgrp); + rd_kafka_mock_sgrp_expire_locks(mshgrp, now); } /** diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 954c0410a2..e48b8f70e9 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -3820,7 +3820,7 @@ rd_kafka_mock_tplist_equal_by_id(rd_kafka_topic_partition_list_t *a, } static rd_kafka_mock_sgrp_partmeta_t * -rd_kafka_mock_sgrp_partmeta_find(rd_kafka_mock_sgrp_t *sgrp, +rd_kafka_mock_sgrp_partmeta_find(rd_kafka_mock_sharegroup_t *sgrp, rd_kafka_Uuid_t topic_id, int32_t partition) { rd_kafka_mock_sgrp_partmeta_t *pmeta; @@ -3836,7 +3836,7 @@ rd_kafka_mock_sgrp_partmeta_find(rd_kafka_mock_sgrp_t *sgrp, } static rd_kafka_mock_sgrp_partmeta_t * -rd_kafka_mock_sgrp_partmeta_get(rd_kafka_mock_sgrp_t *sgrp, +rd_kafka_mock_sgrp_partmeta_get(rd_kafka_mock_sharegroup_t *sgrp, rd_kafka_Uuid_t topic_id, int32_t partition, const rd_kafka_mock_partition_t *mpart) { @@ -4034,7 +4034,7 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_topic_partition_list_t *requested_partitions = NULL; rd_kafka_topic_partition_list_t *forgotten_partitions = NULL; rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR; - rd_kafka_mock_sgrp_t *sgrp = NULL; + rd_kafka_mock_sharegroup_t *sgrp = NULL; rd_kafka_mock_sgrp_fetch_session_t *session = NULL; (void)log_decode_errors; @@ -4166,7 +4166,7 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, * -1 = close session, * >0 = continue (must match expected epoch). */ mtx_lock(&mcluster->lock); - sgrp = rd_kafka_mock_sgrp_get(mcluster, &GroupId); + sgrp = rd_kafka_mock_sharegroup_get(mcluster, &GroupId); /* Look up existing session by MemberId */ TAILQ_FOREACH(session, &sgrp->fetch_sessions, link) { diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index 8db071541e..6d39b04fcb 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -276,6 +276,15 @@ typedef struct rd_kafka_mock_sharegroup_s { members; /**< Share group members */ int member_cnt; /**< Number of share group members */ rd_bool_t manual_assignment; /**< Use manual assignment */ + + /* ShareFetch state (KIP-932) */ + TAILQ_HEAD(, rd_kafka_mock_sgrp_partmeta_s) + partitions; /**< Share partition metadata */ + int partition_cnt; /**< Number of partitions */ + TAILQ_HEAD(, rd_kafka_mock_sgrp_fetch_session_s) + fetch_sessions; /**< Active fetch sessions */ + int fetch_session_cnt; /**< Number of fetch sessions */ + rd_kafka_timer_t fetch_session_tmr; /**< Fetch session expiry timer */ } rd_kafka_mock_sharegroup_t; /** @@ -295,27 +304,6 @@ typedef struct rd_kafka_mock_sharegroup_member_s { rd_kafka_mock_sharegroup_t *mshgrp; /**< Share group */ } rd_kafka_mock_sharegroup_member_t; -/** - * @struct Share group member (KIP-932). - */ -typedef struct rd_kafka_mock_sgrp_member_s { - TAILQ_ENTRY(rd_kafka_mock_sgrp_member_s) link; - char *id; /**< MemberId */ - char *instance_id; /**< Group instance id */ - rd_ts_t ts_last_activity; /**< Last activity */ - int32_t current_member_epoch; /**< Current member epoch */ - int32_t target_member_epoch; /**< Target member epoch */ - rd_kafka_topic_partition_list_t *current_assignment; - rd_kafka_topic_partition_list_t *target_assignment; - rd_kafka_topic_partition_list_t *returned_assignment; - rd_list_t *subscribed_topics; - rd_list_t *subscribed_topic_names; - char *subscribed_topic_regex; - rd_bool_t left_static_membership; - struct rd_kafka_mock_connection_s *conn; - struct rd_kafka_mock_sgrp_s *sgrp; -} rd_kafka_mock_sgrp_member_t; - /** * @brief Share record state. */ @@ -358,23 +346,6 @@ typedef struct rd_kafka_mock_sgrp_fetch_session_s { rd_kafka_topic_partition_list_t *partitions; } rd_kafka_mock_sgrp_fetch_session_t; -/** - * @brief Share group (KIP-932). - */ -typedef struct rd_kafka_mock_sgrp_s { - TAILQ_ENTRY(rd_kafka_mock_sgrp_s) link; - struct rd_kafka_mock_cluster_s *cluster; - char *id; - int32_t group_epoch; - int session_timeout_ms; - rd_kafka_timer_t session_tmr; - TAILQ_HEAD(, rd_kafka_mock_sgrp_member_s) members; - int member_cnt; - TAILQ_HEAD(, rd_kafka_mock_sgrp_partmeta_s) partitions; - int partition_cnt; - TAILQ_HEAD(, rd_kafka_mock_sgrp_fetch_session_s) fetch_sessions; - int fetch_session_cnt; -} rd_kafka_mock_sgrp_t; /** * @struct TransactionalId + PID (+ optional sequence state) @@ -628,7 +599,6 @@ struct rd_kafka_mock_cluster_s { TAILQ_HEAD(, rd_kafka_mock_sgrp_s) sgrps_share; TAILQ_HEAD(, rd_kafka_mock_sharegroup_s) sharegrps; - TAILQ_HEAD(, rd_kafka_mock_sgrp_s) sgrps_share; /** Explicit coordinators (set with mock_set_coordinator()) */ TAILQ_HEAD(, rd_kafka_mock_coord_s) coords; @@ -746,17 +716,13 @@ rd_kafka_mock_topic_t * rd_kafka_mock_topic_find_by_id(const rd_kafka_mock_cluster_t *mcluster, rd_kafka_Uuid_t id); -rd_kafka_mock_sgrp_t * -rd_kafka_mock_sgrp_find(rd_kafka_mock_cluster_t *mcluster, - const rd_kafkap_str_t *GroupId); -rd_kafka_mock_sgrp_t * -rd_kafka_mock_sgrp_get(rd_kafka_mock_cluster_t *mcluster, - const rd_kafkap_str_t *GroupId); -void rd_kafka_mock_sgrp_destroy(rd_kafka_mock_sgrp_t *sgrp); void rd_kafka_mock_sgrp_fetch_session_destroy( rd_kafka_mock_sgrp_fetch_session_t *session); -void rd_kafka_mock_sgrp_release_member_locks(rd_kafka_mock_sgrp_t *sgrp, - const char *member_id); +void rd_kafka_mock_sgrp_release_member_locks( + rd_kafka_mock_sharegroup_t *mshgrp, + const char *member_id); +void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts, + void *arg); rd_kafka_mock_broker_t * rd_kafka_mock_cluster_get_coord(rd_kafka_mock_cluster_t *mcluster, diff --git a/src/rdkafka_mock_sharegrp.c b/src/rdkafka_mock_sharegrp.c index 336d3b1bc9..bd6ed9a8c9 100644 --- a/src/rdkafka_mock_sharegrp.c +++ b/src/rdkafka_mock_sharegrp.c @@ -97,10 +97,21 @@ rd_kafka_mock_sharegroup_get(rd_kafka_mock_cluster_t *mcluster, TAILQ_INIT(&mshgrp->members); mshgrp->member_cnt = 0; + /* ShareFetch state */ + TAILQ_INIT(&mshgrp->partitions); + TAILQ_INIT(&mshgrp->fetch_sessions); + mshgrp->partition_cnt = 0; + mshgrp->fetch_session_cnt = 0; + rd_kafka_timer_start(&mcluster->timers, &mshgrp->session_tmr, 1000 * 1000 /* 1s */, rd_kafka_mock_sharegroup_session_tmr_cb, mshgrp); + /* Fetch session expiry timer */ + rd_kafka_timer_start(&mcluster->timers, &mshgrp->fetch_session_tmr, + 1000 * 1000 /* 1s */, + rd_kafka_mock_sgrp_fetch_session_tmr_cb, mshgrp); + TAILQ_INSERT_TAIL(&mcluster->sharegrps, mshgrp, link); return mshgrp; @@ -111,15 +122,38 @@ rd_kafka_mock_sharegroup_get(rd_kafka_mock_cluster_t *mcluster, */ void rd_kafka_mock_sharegroup_destroy(rd_kafka_mock_sharegroup_t *mshgrp) { rd_kafka_mock_sharegroup_member_t *member; + rd_kafka_mock_sgrp_partmeta_t *pmeta; + rd_kafka_mock_sgrp_fetch_session_t *session; TAILQ_REMOVE(&mshgrp->cluster->sharegrps, mshgrp, link); rd_kafka_timer_stop(&mshgrp->cluster->timers, &mshgrp->session_tmr, RD_DO_LOCK); + rd_kafka_timer_stop(&mshgrp->cluster->timers, + &mshgrp->fetch_session_tmr, RD_DO_LOCK); /* Destroy all members */ while ((member = TAILQ_FIRST(&mshgrp->members))) rd_kafka_mock_sharegroup_member_destroy(mshgrp, member); + /* Destroy ShareFetch partition metadata */ + while ((pmeta = TAILQ_FIRST(&mshgrp->partitions))) { + rd_kafka_mock_sgrp_record_state_t *state; + TAILQ_REMOVE(&mshgrp->partitions, pmeta, link); + while ((state = TAILQ_FIRST(&pmeta->inflight))) { + TAILQ_REMOVE(&pmeta->inflight, state, link); + rd_free(state->owner_member_id); + rd_free(state); + } + rd_free(pmeta); + } + + /* Destroy ShareFetch sessions */ + while ((session = TAILQ_FIRST(&mshgrp->fetch_sessions))) { + TAILQ_REMOVE(&mshgrp->fetch_sessions, session, link); + mshgrp->fetch_session_cnt--; + rd_kafka_mock_sgrp_fetch_session_destroy(session); + } + rd_free(mshgrp->id); rd_free(mshgrp); } diff --git a/tests/0152-kip932-sharefetch_mockbroker.c b/tests/0152-kip932-sharefetch_mockbroker.c index 08662f718b..e56e688743 100644 --- a/tests/0152-kip932-sharefetch_mockbroker.c +++ b/tests/0152-kip932-sharefetch_mockbroker.c @@ -413,8 +413,9 @@ static int run_sharefetch_session_expiry_rtt(void) { rd_kafka_share_t *consumer; int consumed; - rd_kafka_mock_set_group_consumer_session_timeout_ms(ctx.mcluster, 200); - rd_kafka_mock_broker_set_rtt(ctx.mcluster, 1, 500); + /* Session timeout must be long enough for normal requests + * to complete, but short enough to expire during high RTT. */ + rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 1000); if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != RD_KAFKA_RESP_ERR_NO_ERROR) @@ -424,9 +425,22 @@ static int run_sharefetch_session_expiry_rtt(void) { consumer = new_share_consumer(ctx.bootstraps, "sg-rtt-expiry"); subscribe_topics(consumer, &topic, 1); + /* Phase 1: consume one message with normal RTT (no injection). */ consumed = consume_n(consumer, 1, 20); - usleep(1000 * 1000); /* allow session timeout to elapse */ - consumed += consume_n(consumer, 1, 20); + printf(" rtt_expiry: phase1 consumed %d/1\n", consumed); + + /* Phase 2: inject RTT >> session timeout to force session expiry. + * All requests to broker 1 now take 3s, but the session + * expires after 1s of inactivity. */ + rd_kafka_mock_broker_set_rtt(ctx.mcluster, 1, 3000); + usleep(2000 * 1000); /* wait for session to expire */ + + /* Phase 3: clear RTT and let the consumer recover. + * It should re-create the session (epoch=0) and consume + * the remaining message. */ + rd_kafka_mock_broker_set_rtt(ctx.mcluster, 1, 0); + consumed += consume_n(consumer, 1, 30); + printf(" rtt_expiry: phase3 consumed %d/2 total\n", consumed); rd_kafka_share_consumer_close(consumer); rd_kafka_share_destroy(consumer); @@ -497,7 +511,7 @@ static int run_multi_consumer_lock_expiry(void) { int consumed_a, consumed_b; /* Use a short session/lock timeout so the test runs quickly. */ - rd_kafka_mock_set_group_consumer_session_timeout_ms(ctx.mcluster, 500); + rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 500); if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != RD_KAFKA_RESP_ERR_NO_ERROR) From 83e7a7a529be5bc0a2d76e39c3410a179e0a3a25 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Tue, 10 Feb 2026 01:10:45 +0530 Subject: [PATCH 13/20] Address feedbacks --- src/rdkafka_mock_handlers.c | 98 +++++++++++++++-------- tests/0152-kip932-sharefetch_mockbroker.c | 44 ++++++++++ 2 files changed, 107 insertions(+), 35 deletions(-) diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index e48b8f70e9..a7f920ff26 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -1461,24 +1461,6 @@ static int rd_kafka_mock_handle_Metadata(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_write_i32(resp, INT32_MIN); } - /* ForgottenToppars */ - { - int32_t ForgottenTopicsCnt; - rd_kafka_buf_read_arraycnt(rkbuf, &ForgottenTopicsCnt, - RD_KAFKAP_TOPICS_MAX); - while (ForgottenTopicsCnt-- > 0) { - rd_kafka_Uuid_t TopicId = RD_KAFKA_UUID_ZERO; - int32_t PartitionCnt; - rd_kafka_buf_read_uuid(rkbuf, &TopicId); - rd_kafka_buf_read_arraycnt(rkbuf, &PartitionCnt, - RD_KAFKAP_PARTITIONS_MAX); - while (PartitionCnt-- > 0) { - rd_kafka_buf_read_i32(rkbuf, NULL); /* Partition */ - } - rd_kafka_buf_skip_tags(rkbuf); /* Topic tags */ - } - } - if (rd_kafka_buf_read_remain(rkbuf) > 0) rd_kafka_buf_skip_tags(rkbuf); rd_kafka_buf_write_tags_empty(resp); @@ -3979,7 +3961,7 @@ rd_kafka_mock_sgrp_partmeta_prune_archived(rd_kafka_mock_sgrp_partmeta_t *pmeta) TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { if (state->state != RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED) continue; - if (state->offset <= pmeta->speo) + if (state->offset >= pmeta->spso) continue; TAILQ_REMOVE(&pmeta->inflight, state, link); @@ -3989,36 +3971,82 @@ rd_kafka_mock_sgrp_partmeta_prune_archived(rd_kafka_mock_sgrp_partmeta_t *pmeta) } } -static const rd_kafka_mock_msgset_t * -rd_kafka_mock_sgrp_first_acquired_msgset( +/** + * @brief Write all acquired record batches for the given partition and member + * into the response buffer as a single Records field (compact bytes + * containing concatenated RecordBatches). + * + * @returns the total number of record data bytes written (0 if none). + */ +static size_t +rd_kafka_mock_sgrp_write_acquired_records( + rd_kafka_buf_t *resp, rd_kafka_mock_sgrp_partmeta_t *pmeta, const rd_kafka_mock_partition_t *mpart, const rd_kafkap_str_t *member_id, rd_ts_t now) { + const rd_kafka_mock_msgset_t *msgsets[256]; + int msgset_cnt = 0; int64_t offset; + size_t total_len = 0; + int i; + /* Collect unique msgsets containing acquired records for this member */ for (offset = pmeta->spso; offset <= pmeta->speo; offset++) { rd_kafka_mock_sgrp_record_state_t *state = rd_kafka_mock_sgrp_record_state_find(pmeta, offset); const rd_kafka_mock_msgset_t *mset; + int j, dup; if (!state || state->state != RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) continue; - if (state->lock_expiry_ts && - state->lock_expiry_ts <= now) + if (state->lock_expiry_ts && state->lock_expiry_ts <= now) continue; if (rd_kafkap_str_cmp_str(member_id, state->owner_member_id)) continue; mset = rd_kafka_mock_msgset_find(mpart, offset, rd_false); - if (mset) - return mset; + if (!mset) + continue; + + /* Deduplicate: multiple offsets may fall in the same batch */ + dup = 0; + for (j = 0; j < msgset_cnt; j++) { + if (msgsets[j] == mset) { + dup = 1; + break; + } + } + if (dup) + continue; + + if (msgset_cnt >= + (int)(sizeof(msgsets) / sizeof(msgsets[0]))) + break; /* safety cap */ + + msgsets[msgset_cnt++] = mset; + total_len += RD_KAFKAP_BYTES_LEN(&mset->bytes); } - return NULL; + if (msgset_cnt == 0) { + /* No acquired records: write NULL compact bytes */ + rd_kafka_buf_write_kbytes(resp, NULL); + return 0; + } + + /* Write compact bytes length prefix (N+1 encoding) */ + rd_kafka_buf_write_uvarint(resp, (uint64_t)(total_len + 1)); + + /* Write each msgset's raw bytes back-to-back */ + for (i = 0; i < msgset_cnt; i++) { + rd_kafka_buf_write(resp, msgsets[i]->bytes.data, + RD_KAFKAP_BYTES_LEN(&msgsets[i]->bytes)); + } + + return total_len; } static int @@ -4425,12 +4453,6 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, sgrp, topic_id, rktpar->partition) : NULL; - const rd_kafka_mock_msgset_t *mset = - (mpart && pmeta) - ? rd_kafka_mock_sgrp_first_acquired_msgset( - pmeta, mpart, &MemberId, - now) - : NULL; rd_kafka_resp_err_t part_err = mpart ? RD_KAFKA_RESP_ERR_NO_ERROR : RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; @@ -4457,9 +4479,15 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_buf_write_i32(resp, -1); rd_kafka_buf_write_i32(resp, -1); rd_kafka_buf_write_tags_empty(resp); - /* Response: Records */ - rd_kafka_buf_write_kbytes( - resp, mset ? &mset->bytes : NULL); + /* Response: Records (all acquired + * batches concatenated) */ + if (mpart && pmeta) + rd_kafka_mock_sgrp_write_acquired_records( + resp, pmeta, mpart, + &MemberId, now); + else + rd_kafka_buf_write_kbytes( + resp, NULL); /* Response: AcquiredRecords */ rd_kafka_buf_write_arraycnt(resp, 0); /* Response: Partition tags */ diff --git a/tests/0152-kip932-sharefetch_mockbroker.c b/tests/0152-kip932-sharefetch_mockbroker.c index e56e688743..7af9556318 100644 --- a/tests/0152-kip932-sharefetch_mockbroker.c +++ b/tests/0152-kip932-sharefetch_mockbroker.c @@ -503,6 +503,49 @@ static int run_forgotten_topics(void) { * After the lock expiry timeout, consumer B should be able to pick up * the same records because the proactive lock-expiry scan releases them. */ +/** + * @brief Produce messages one-at-a-time (each flush creates a separate + * msgset on the mock partition), then consume and verify all are + * received. This validates that the ShareFetch response includes + * records from *all* acquired msgsets, not just the first one. + */ +static int run_multi_batch_consume(void) { + const char *topic = "kip932_multi_batch"; + const int msgcnt = 5; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed; + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + + /* Produce each message individually with a flush in between, + * guaranteeing separate msgsets on the mock partition. */ + for (int i = 0; i < msgcnt; i++) { + char payload[64]; + snprintf(payload, sizeof(payload), "batch-%d", i); + if (rd_kafka_producev( + ctx.producer, RD_KAFKA_V_TOPIC(topic), + RD_KAFKA_V_VALUE(payload, strlen(payload)), + RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY), + RD_KAFKA_V_END) != RD_KAFKA_RESP_ERR_NO_ERROR) + die("Produce failed"); + rd_kafka_flush(ctx.producer, 5000); + } + + consumer = new_share_consumer(ctx.bootstraps, "sg-multi-batch"); + subscribe_topics(consumer, &topic, 1); + consumed = consume_n(consumer, msgcnt, 50); + printf(" multi_batch: consumed %d/%d\n", consumed, msgcnt); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed == msgcnt; +} + static int run_multi_consumer_lock_expiry(void) { const char *topic = "kip932_multi_consumer_lock"; const int msgcnt = 3; @@ -568,6 +611,7 @@ int main(void) { failures += run_test("sharefetch_session_expiry_rtt", run_sharefetch_session_expiry_rtt); failures += run_test("forgotten_topics", run_forgotten_topics); + failures += run_test("multi_batch_consume", run_multi_batch_consume); failures += run_test("multi_consumer_lock_expiry", run_multi_consumer_lock_expiry); From 9b95db51f692d4bf8498c2ed9e504a0b31b2a42e Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Tue, 10 Feb 2026 01:38:10 +0530 Subject: [PATCH 14/20] Implement max delivery attempts and record lock --- src/rdkafka_mock.h | 26 +++++ src/rdkafka_mock_handlers.c | 19 +++- src/rdkafka_mock_int.h | 14 +++ src/rdkafka_mock_sharegrp.c | 41 +++++++- tests/0152-kip932-sharefetch_mockbroker.c | 111 ++++++++++++++++++++++ 5 files changed, 208 insertions(+), 3 deletions(-) diff --git a/src/rdkafka_mock.h b/src/rdkafka_mock.h index 18eb030770..fa9785e697 100644 --- a/src/rdkafka_mock.h +++ b/src/rdkafka_mock.h @@ -641,6 +641,32 @@ RD_EXPORT void rd_kafka_mock_sharegroup_set_heartbeat_interval( rd_kafka_mock_cluster_t *mcluster, int heartbeat_interval_ms); +/** + * @brief Set the maximum number of times a record can be acquired + * before it is automatically archived (dead-lettered). + * + * Default is 5. Set to 0 for unlimited deliveries. + * + * @param mcluster Mock cluster instance. + * @param max_attempts Maximum delivery attempts per record. + */ +RD_EXPORT void rd_kafka_mock_sharegroup_set_max_delivery_attempts( + rd_kafka_mock_cluster_t *mcluster, + int max_attempts); + +/** + * @brief Set the per-record lock duration in milliseconds. + * + * When a record is acquired, its lock expires after this duration. + * Default is 0, which falls back to the session timeout value. + * + * @param mcluster Mock cluster instance. + * @param lock_duration_ms Lock duration in milliseconds. + */ +RD_EXPORT void rd_kafka_mock_sharegroup_set_record_lock_duration( + rd_kafka_mock_cluster_t *mcluster, + int lock_duration_ms); + /** * @brief Set a manual target assignment for a sharegroup. * diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index a7f920ff26..c3f1dc0366 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -3908,6 +3908,7 @@ rd_kafka_mock_sgrp_acquire_available_offsets( const rd_kafka_mock_partition_t *mpart, const rd_kafkap_str_t *member_id, rd_ts_t lock_expiry_ts, + int max_delivery_attempts, int64_t *remaining_records, int64_t *remaining_bytes, int *acquired_cnt, @@ -3929,6 +3930,18 @@ rd_kafka_mock_sgrp_acquire_available_offsets( state->state != RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE) continue; + /* Check max delivery attempts: if the record has already + * been acquired (and released/expired) too many times, + * archive it instead of re-acquiring. */ + if (max_delivery_attempts > 0 && state && + state->delivery_count >= max_delivery_attempts) { + state->state = RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED; + RD_IF_FREE(state->owner_member_id, rd_free); + state->owner_member_id = NULL; + state->lock_expiry_ts = 0; + continue; + } + mset = rd_kafka_mock_msgset_find(mpart, offset, rd_false); if (!mset) continue; @@ -4362,7 +4375,11 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, pmeta, mpart, &MemberId, now + - (sgrp->session_timeout_ms * 1000), + ((sgrp->record_lock_duration_ms > 0 + ? sgrp->record_lock_duration_ms + : sgrp->session_timeout_ms) * + 1000), + sgrp->max_delivery_attempts, MaxRecords > 0 ? &remaining_records : NULL, MaxBytes > 0 ? &remaining_bytes : NULL, &acquired_cnt, &acquired_bytes); diff --git a/src/rdkafka_mock_int.h b/src/rdkafka_mock_int.h index 6d39b04fcb..bc0c270ef7 100644 --- a/src/rdkafka_mock_int.h +++ b/src/rdkafka_mock_int.h @@ -285,6 +285,14 @@ typedef struct rd_kafka_mock_sharegroup_s { fetch_sessions; /**< Active fetch sessions */ int fetch_session_cnt; /**< Number of fetch sessions */ rd_kafka_timer_t fetch_session_tmr; /**< Fetch session expiry timer */ + + /* Per-record limits */ + int max_delivery_attempts; /**< Max times a record can be + * acquired before being archived. + * 0 = unlimited (default 5). */ + int record_lock_duration_ms; /**< Per-record lock duration in ms. + * 0 = use session_timeout_ms + * as fallback (default 0). */ } rd_kafka_mock_sharegroup_t; /** @@ -642,6 +650,12 @@ struct rd_kafka_mock_cluster_s { int sharegroup_session_timeout_ms; /** Heartbeat interval (KIP 932) */ int sharegroup_heartbeat_interval_ms; + /** Max delivery attempts per record (KIP 932). + * 0 = unlimited. */ + int sharegroup_max_delivery_attempts; + /** Per-record lock duration in ms (KIP 932). + * 0 = use session_timeout_ms. */ + int sharegroup_record_lock_duration_ms; } defaults; /**< Dynamic array of IO handlers for corresponding fd in .fds */ diff --git a/src/rdkafka_mock_sharegrp.c b/src/rdkafka_mock_sharegrp.c index bd6ed9a8c9..dc51256c36 100644 --- a/src/rdkafka_mock_sharegrp.c +++ b/src/rdkafka_mock_sharegrp.c @@ -53,8 +53,10 @@ static void rd_kafka_mock_sharegroup_session_tmr_cb(rd_kafka_timers_t *rkts, */ void rd_kafka_mock_sharegrps_init(rd_kafka_mock_cluster_t *mcluster) { TAILQ_INIT(&mcluster->sharegrps); - mcluster->defaults.sharegroup_session_timeout_ms = 45000; - mcluster->defaults.sharegroup_heartbeat_interval_ms = 5000; + mcluster->defaults.sharegroup_session_timeout_ms = 45000; + mcluster->defaults.sharegroup_heartbeat_interval_ms = 5000; + mcluster->defaults.sharegroup_max_delivery_attempts = 5; + mcluster->defaults.sharegroup_record_lock_duration_ms = 0; } /** @@ -103,6 +105,12 @@ rd_kafka_mock_sharegroup_get(rd_kafka_mock_cluster_t *mcluster, mshgrp->partition_cnt = 0; mshgrp->fetch_session_cnt = 0; + /* Per-record limits */ + mshgrp->max_delivery_attempts = + mcluster->defaults.sharegroup_max_delivery_attempts; + mshgrp->record_lock_duration_ms = + mcluster->defaults.sharegroup_record_lock_duration_ms; + rd_kafka_timer_start(&mcluster->timers, &mshgrp->session_tmr, 1000 * 1000 /* 1s */, rd_kafka_mock_sharegroup_session_tmr_cb, mshgrp); @@ -708,6 +716,35 @@ void rd_kafka_mock_sharegroup_set_heartbeat_interval( mtx_unlock(&mcluster->lock); } +/** + * @brief Set the maximum delivery attempts per record for the sharegroup. + */ +void rd_kafka_mock_sharegroup_set_max_delivery_attempts( + rd_kafka_mock_cluster_t *mcluster, + int max_attempts) { + rd_kafka_mock_sharegroup_t *mshgrp; + mtx_lock(&mcluster->lock); + TAILQ_FOREACH(mshgrp, &mcluster->sharegrps, link) + mshgrp->max_delivery_attempts = max_attempts; + mcluster->defaults.sharegroup_max_delivery_attempts = max_attempts; + mtx_unlock(&mcluster->lock); +} + +/** + * @brief Set the per-record lock duration in milliseconds for the sharegroup. + */ +void rd_kafka_mock_sharegroup_set_record_lock_duration( + rd_kafka_mock_cluster_t *mcluster, + int lock_duration_ms) { + rd_kafka_mock_sharegroup_t *mshgrp; + mtx_lock(&mcluster->lock); + TAILQ_FOREACH(mshgrp, &mcluster->sharegrps, link) + mshgrp->record_lock_duration_ms = lock_duration_ms; + mcluster->defaults.sharegroup_record_lock_duration_ms = + lock_duration_ms; + mtx_unlock(&mcluster->lock); +} + /** * @brief A client connection closed, check if any sharegroup has any * state for this connection that needs to be cleared. diff --git a/tests/0152-kip932-sharefetch_mockbroker.c b/tests/0152-kip932-sharefetch_mockbroker.c index 7af9556318..0886739ff0 100644 --- a/tests/0152-kip932-sharefetch_mockbroker.c +++ b/tests/0152-kip932-sharefetch_mockbroker.c @@ -546,6 +546,113 @@ static int run_multi_batch_consume(void) { return consumed == msgcnt; } +/** + * @brief Verify that max_delivery_attempts causes records to be archived + * after the limit is exceeded. Consumer A acquires all records, then + * its session times out (releasing locks). Consumer B acquires them + * again, and its session also times out. After the delivery limit is + * exhausted, Consumer C should see 0 available records. + */ +static int run_max_delivery_attempts(void) { + const char *topic = "kip932_max_delivery"; + const int msgcnt = 3; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed_a, consumed_b, consumed_c; + + /* Set max delivery attempts to 2 and a short session timeout + * so locks expire quickly after consumer destruction. */ + rd_kafka_mock_sharegroup_set_max_delivery_attempts(ctx.mcluster, 2); + rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 500); + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, msgcnt); + + /* Delivery 1: Consumer A acquires and "crashes" (no ack). */ + consumer = new_share_consumer(ctx.bootstraps, "sg-max-delivery"); + subscribe_topics(consumer, &topic, 1); + consumed_a = consume_n(consumer, msgcnt, 50); + printf(" max_delivery: A consumed %d/%d (delivery 1)\n", + consumed_a, msgcnt); + rd_kafka_share_destroy(consumer); + usleep(1500 * 1000); /* wait for lock expiry */ + + /* Delivery 2: Consumer B acquires same records again (delivery_count + * reaches 2 = limit) and "crashes". */ + consumer = new_share_consumer(ctx.bootstraps, "sg-max-delivery"); + subscribe_topics(consumer, &topic, 1); + consumed_b = consume_n(consumer, msgcnt, 50); + printf(" max_delivery: B consumed %d/%d (delivery 2)\n", + consumed_b, msgcnt); + rd_kafka_share_destroy(consumer); + usleep(1500 * 1000); /* wait for lock expiry */ + + /* Delivery 3 attempt: Consumer C should get 0 records because + * all records have been archived (delivery_count >= max). */ + consumer = new_share_consumer(ctx.bootstraps, "sg-max-delivery"); + subscribe_topics(consumer, &topic, 1); + consumed_c = consume_n(consumer, 1, 10); + printf(" max_delivery: C consumed %d/0 (should be archived)\n", + consumed_c); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed_a == msgcnt && consumed_b == msgcnt && consumed_c == 0; +} + +/** + * @brief Verify that record_lock_duration_ms controls how long acquired + * records stay locked, independently of session_timeout_ms. + * Sets a short lock duration (300ms) with a longer session timeout + * (10s). Consumer A acquires records and "crashes". After the short + * lock duration expires, Consumer B should be able to acquire them + * even though A's session hasn't timed out yet. + */ +static int run_record_lock_duration(void) { + const char *topic = "kip932_lock_duration"; + const int msgcnt = 3; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed_a, consumed_b; + + /* Long session timeout, short record lock duration. */ + rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 10000); + rd_kafka_mock_sharegroup_set_record_lock_duration(ctx.mcluster, 300); + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, msgcnt); + + /* Consumer A acquires records, then crashes (no close). */ + consumer = new_share_consumer(ctx.bootstraps, "sg-lock-duration"); + subscribe_topics(consumer, &topic, 1); + consumed_a = consume_n(consumer, msgcnt, 50); + printf(" lock_duration: A consumed %d/%d\n", consumed_a, msgcnt); + rd_kafka_share_destroy(consumer); + + /* Wait for record lock to expire (300ms + margin), + * but NOT session timeout (10s). */ + usleep(800 * 1000); + + /* Consumer B should get the records because locks have expired + * even though A's session is still technically alive. */ + consumer = new_share_consumer(ctx.bootstraps, "sg-lock-duration"); + subscribe_topics(consumer, &topic, 1); + consumed_b = consume_n(consumer, msgcnt, 50); + printf(" lock_duration: B consumed %d/%d\n", consumed_b, msgcnt); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed_a == msgcnt && consumed_b == msgcnt; +} + static int run_multi_consumer_lock_expiry(void) { const char *topic = "kip932_multi_consumer_lock"; const int msgcnt = 3; @@ -612,6 +719,10 @@ int main(void) { run_sharefetch_session_expiry_rtt); failures += run_test("forgotten_topics", run_forgotten_topics); failures += run_test("multi_batch_consume", run_multi_batch_consume); + failures += run_test("max_delivery_attempts", + run_max_delivery_attempts); + failures += run_test("record_lock_duration", + run_record_lock_duration); failures += run_test("multi_consumer_lock_expiry", run_multi_consumer_lock_expiry); From c98ffa3f0111bdf8d288f24f772f5cb39b446d74 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Tue, 10 Feb 2026 01:57:01 +0530 Subject: [PATCH 15/20] Add member validation for share fetch --- src/rdkafka_mock_handlers.c | 19 +++++- tests/0152-kip932-sharefetch_mockbroker.c | 80 +++++++++++++++++++++++ 2 files changed, 96 insertions(+), 3 deletions(-) diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index c3f1dc0366..14c242c349 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -4209,14 +4209,27 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, mtx_lock(&mcluster->lock); sgrp = rd_kafka_mock_sharegroup_get(mcluster, &GroupId); + /* Member validation: verify the MemberId is a registered + * member of this share group (joined via SGHB). */ + if (!rd_kafka_mock_sharegroup_member_find(sgrp, &MemberId)) { + err = RD_KAFKA_RESP_ERR_UNKNOWN_MEMBER_ID; + rd_kafka_dbg( + mconn->broker->cluster->rk, MOCK, "MOCK", + "ShareFetch: unknown member %.*s in group %.*s", + RD_KAFKAP_STR_PR(&MemberId), + RD_KAFKAP_STR_PR(&GroupId)); + } + /* Look up existing session by MemberId */ + if (!err) { TAILQ_FOREACH(session, &sgrp->fetch_sessions, link) { if (!rd_kafkap_str_cmp_str(&MemberId, session->member_id)) break; } + } - if (SessionEpoch == 0) { + if (!err && SessionEpoch == 0) { /* Open a new session (or reuse if one already exists * for this member). */ if (!session) { @@ -4241,7 +4254,7 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, rd_kafka_topic_partition_list_copy( requested_partitions); } - } else if (SessionEpoch == -1) { + } else if (!err && SessionEpoch == -1) { /* Close the session and release all locks held * by this member. */ if (session) { @@ -4256,7 +4269,7 @@ rd_kafka_mock_handle_ShareFetch(rd_kafka_mock_connection_t *mconn, } /* Closing a non-existent session is not an error; * proceed with an empty response. */ - } else { + } else if (!err) { /* Continue existing session: validate epoch. */ if (!session) { err = diff --git a/tests/0152-kip932-sharefetch_mockbroker.c b/tests/0152-kip932-sharefetch_mockbroker.c index 0886739ff0..1e9c07768f 100644 --- a/tests/0152-kip932-sharefetch_mockbroker.c +++ b/tests/0152-kip932-sharefetch_mockbroker.c @@ -407,6 +407,84 @@ static int run_empty_fetch_no_records(void) { return consumed == 0; } +/** + * @brief Verify that ShareFetch rejects requests from an unregistered member + * (UNKNOWN_MEMBER_ID), and that after the member re-joins it can + * consume again. + * + * Phase 1: Consumer joins normally via SGHB → consumes messages OK. + * Phase 2: Push SGHB errors → heartbeats fail → member expires → broker + * rejects ShareFetch with UNKNOWN_MEMBER_ID. + * Phase 3: SGHB errors drain → member re-joins → consumes again. + */ +static int run_member_validation(void) { + const char *topic = "kip932_member_validation"; + const int msgcnt = 4; + test_ctx_t ctx = test_ctx_new(); + rd_kafka_share_t *consumer; + int consumed_p1, consumed_p3; + + /* Short session timeout so the member is evicted quickly once + * heartbeats stop succeeding. */ + rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 500); + + if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != + RD_KAFKA_RESP_ERR_NO_ERROR) + die("Failed to create mock topic"); + produce_messages(ctx.producer, topic, msgcnt); + + consumer = new_share_consumer(ctx.bootstraps, "sg-member-val"); + subscribe_topics(consumer, &topic, 1); + + /* Phase 1: Consume normally — member is registered via SGHB. */ + consumed_p1 = consume_n(consumer, 2, 30); + printf(" member_validation: phase1 consumed %d/2\n", consumed_p1); + + /* Phase 2: Block SGHB so heartbeats fail. + * Push enough errors to cover the window while we wait for the + * member to be evicted. The client sends SGHB roughly every + * heartbeat interval (~5s default), but with a 500ms session + * timeout the member will be removed well before the errors + * drain. We push 20 errors to be safe. */ + rd_kafka_mock_push_request_errors( + ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, 20, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, + RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE); + + /* Wait for the member to be evicted (500ms session timeout + margin). */ + usleep(1500 * 1000); + + /* Phase 3: SGHB errors will eventually drain. Once a SGHB + * succeeds, the member re-joins and the remaining records + * become fetchable again. */ + consumed_p3 = consume_n(consumer, 2, 50); + printf(" member_validation: phase3 consumed %d/2\n", consumed_p3); + + rd_kafka_share_consumer_close(consumer); + rd_kafka_share_destroy(consumer); + test_ctx_destroy(&ctx); + + return consumed_p1 >= 2 && (consumed_p1 + consumed_p3) >= msgcnt; +} + static int run_sharefetch_session_expiry_rtt(void) { const char *topic = "kip932_rtt_expiry"; test_ctx_t ctx = test_ctx_new(); @@ -739,6 +817,8 @@ int main(void) { run_unknown_topic_subscription); failures += run_test("empty_fetch_no_records", run_empty_fetch_no_records); + failures += run_test("member_validation", + run_member_validation); printf("Failures: %d\n", failures); return failures ? 1 : 0; From 15080773bc4bc38e1db35e9be338929f8692c7da Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Tue, 10 Feb 2026 11:53:12 +0530 Subject: [PATCH 16/20] Remove tmp produce code --- src/rdkafka_mock.c | 68 ---------------------------------------------- src/rdkafka_mock.h | 20 -------------- src/rdkafka_op.c | 2 -- src/rdkafka_op.h | 10 ------- 4 files changed, 100 deletions(-) diff --git a/src/rdkafka_mock.c b/src/rdkafka_mock.c index 3c8b00988f..ac9674c911 100644 --- a/src/rdkafka_mock.c +++ b/src/rdkafka_mock.c @@ -2223,47 +2223,6 @@ rd_kafka_mock_partition_set_follower_wmarks(rd_kafka_mock_cluster_t *mcluster, rd_kafka_op_req(mcluster->ops, rko, RD_POLL_INFINITE)); } -rd_kafka_resp_err_t -rd_kafka_mock_partition_produce(rd_kafka_mock_cluster_t *mcluster, - const char *topic, - int32_t partition, - const void *records, - size_t records_size, - const char *transactional_id, - int64_t *base_offset) { - rd_kafka_op_t *rko; - rd_kafka_op_t *reply; - - if (!mcluster || !topic || partition < 0 || !records || - records_size == 0 || records_size > INT32_MAX) { - if (base_offset) - *base_offset = -1; - return RD_KAFKA_RESP_ERR__INVALID_ARG; - } - - rko = rd_kafka_op_new(RD_KAFKA_OP_MOCK); - rko->rko_u.mock.name = rd_strdup(topic); - rko->rko_u.mock.partition = partition; - rko->rko_u.mock.records = - rd_kafkap_bytes_new(records, (int32_t)records_size); - rko->rko_u.mock.base_offset = -1; - if (transactional_id) - rko->rko_u.mock.str = rd_strdup(transactional_id); - rko->rko_u.mock.cmd = RD_KAFKA_MOCK_CMD_PART_PRODUCE; - - reply = rd_kafka_op_req(mcluster->ops, rko, RD_POLL_INFINITE); - if (!reply) { - if (base_offset) - *base_offset = -1; - return RD_KAFKA_RESP_ERR__TIMED_OUT; - } - - if (base_offset) - *base_offset = reply->rko_u.mock.base_offset; - - return rd_kafka_op_err_destroy(reply); -} - rd_kafka_resp_err_t rd_kafka_mock_partition_push_leader_response(rd_kafka_mock_cluster_t *mcluster, const char *topic, @@ -2629,33 +2588,6 @@ rd_kafka_mock_cluster_cmd(rd_kafka_mock_cluster_t *mcluster, mpart->update_follower_end_offset = rd_false; } break; - case RD_KAFKA_MOCK_CMD_PART_PRODUCE: { - rd_kafkap_str_t TransactionalId = RD_KAFKAP_STR_INITIALIZER; - - mpart = rd_kafka_mock_partition_get( - mcluster, rko->rko_u.mock.name, rko->rko_u.mock.partition); - if (!mpart) - return RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART; - if (!mpart->leader) - return RD_KAFKA_RESP_ERR_LEADER_NOT_AVAILABLE; - - if (!rko->rko_u.mock.records || - RD_KAFKAP_BYTES_IS_NULL(rko->rko_u.mock.records) || - RD_KAFKAP_BYTES_LEN(rko->rko_u.mock.records) == 0) - return RD_KAFKA_RESP_ERR__INVALID_ARG; - - if (rko->rko_u.mock.str && rko->rko_u.mock.str[0]) { - TransactionalId.str = rko->rko_u.mock.str; - TransactionalId.len = - (int)strlen(rko->rko_u.mock.str); - } - - rko->rko_u.mock.base_offset = -1; - err = rd_kafka_mock_partition_log_append( - mpart, rko->rko_u.mock.records, &TransactionalId, - &rko->rko_u.mock.base_offset); - return err; - } case RD_KAFKA_MOCK_CMD_PART_PUSH_LEADER_RESPONSE: mpart = rd_kafka_mock_partition_get( mcluster, rko->rko_u.mock.name, rko->rko_u.mock.partition); diff --git a/src/rdkafka_mock.h b/src/rdkafka_mock.h index fa9785e697..f42fb4b528 100644 --- a/src/rdkafka_mock.h +++ b/src/rdkafka_mock.h @@ -292,26 +292,6 @@ rd_kafka_mock_partition_set_follower_wmarks(rd_kafka_mock_cluster_t *mcluster, int64_t lo, int64_t hi); -/** - * @brief Append a record batch to a partition log. - * - * The record batch must use the Kafka record batch format (v2), the same - * encoding as the ProduceRequest Records field. - * - * The topic will be created if it does not exist. - * - * @param transactional_id Optional transactional.id to validate PID/sequence. - * @param base_offset Optional pointer to store the first assigned offset. - */ -RD_EXPORT rd_kafka_resp_err_t -rd_kafka_mock_partition_produce(rd_kafka_mock_cluster_t *mcluster, - const char *topic, - int32_t partition, - const void *records, - size_t records_size, - const char *transactional_id, - int64_t *base_offset); - /** * @brief Push \p cnt Metadata leader response * onto the cluster's stack for the given \p topic and \p partition. diff --git a/src/rdkafka_op.c b/src/rdkafka_op.c index e15593cf1e..0cce441768 100644 --- a/src/rdkafka_op.c +++ b/src/rdkafka_op.c @@ -481,8 +481,6 @@ void rd_kafka_op_destroy(rd_kafka_op_t *rko) { case RD_KAFKA_OP_MOCK: RD_IF_FREE(rko->rko_u.mock.name, rd_free); RD_IF_FREE(rko->rko_u.mock.str, rd_free); - if (rko->rko_u.mock.records) - rd_kafkap_bytes_destroy(rko->rko_u.mock.records); if (rko->rko_u.mock.metrics) { int64_t i; for (i = 0; i < rko->rko_u.mock.hi; i++) diff --git a/src/rdkafka_op.h b/src/rdkafka_op.h index 2a78018183..6d584609f6 100644 --- a/src/rdkafka_op.h +++ b/src/rdkafka_op.h @@ -591,7 +591,6 @@ struct rd_kafka_op_s { RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER, RD_KAFKA_MOCK_CMD_PART_SET_FOLLOWER_WMARKS, RD_KAFKA_MOCK_CMD_PART_PUSH_LEADER_RESPONSE, - RD_KAFKA_MOCK_CMD_PART_PRODUCE, RD_KAFKA_MOCK_CMD_BROKER_SET_UPDOWN, RD_KAFKA_MOCK_CMD_BROKER_SET_RTT, RD_KAFKA_MOCK_CMD_BROKER_SET_RACK, @@ -613,11 +612,9 @@ struct rd_kafka_op_s { * BROKER_SET_RACK * COORD_SET (key_type) * PART_PUSH_LEADER_RESPONSE - * PART_PRODUCE */ char *str; /**< For: * COORD_SET (key) - * PART_PRODUCE */ int32_t partition; /**< For: * PART_SET_FOLLOWER @@ -625,7 +622,6 @@ struct rd_kafka_op_s { * PART_SET_LEADER * APIVERSION_SET (ApiKey) * PART_PUSH_LEADER_RESPONSE - * PART_PRODUCE */ int32_t broker_id; /**< For: * PART_SET_FOLLOWER @@ -657,12 +653,6 @@ struct rd_kafka_op_s { */ char **metrics; /**< Metrics requested, for: * REQUESTED_METRICS_SET */ - rd_kafkap_bytes_t *records; /**< Records, for: - * PART_PRODUCE - */ - int64_t base_offset; /**< BaseOffset for: - * PART_PRODUCE reply - */ } mock; struct { From 97f95465194f6e43418d14debcedfdc064ac691a Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Wed, 11 Feb 2026 16:11:56 +0530 Subject: [PATCH 17/20] Implement feedback --- src/rdkafka_mock_cgrp.c | 120 ------------------------------------ src/rdkafka_mock_handlers.c | 33 ++++------ src/rdkafka_mock_sharegrp.c | 117 +++++++++++++++++++++++++++++++++++ 3 files changed, 130 insertions(+), 140 deletions(-) diff --git a/src/rdkafka_mock_cgrp.c b/src/rdkafka_mock_cgrp.c index 23177d2a0e..28978a56ec 100644 --- a/src/rdkafka_mock_cgrp.c +++ b/src/rdkafka_mock_cgrp.c @@ -1862,126 +1862,6 @@ void rd_kafka_mock_cgrp_consumer_destroy(rd_kafka_mock_cgrp_consumer_t *mcgrp) { rd_free(mcgrp); } -/** - * @brief Destroy share-partition metadata. - */ -static void -rd_kafka_mock_sgrp_partmeta_destroy(rd_kafka_mock_sgrp_partmeta_t *pmeta) { - rd_kafka_mock_sgrp_record_state_t *state; - while ((state = TAILQ_FIRST(&pmeta->inflight))) { - TAILQ_REMOVE(&pmeta->inflight, state, link); - rd_free(state->owner_member_id); - rd_free(state); - } - rd_free(pmeta); -} - -/** - * @brief Destroy share fetch session. - */ -void rd_kafka_mock_sgrp_fetch_session_destroy( - rd_kafka_mock_sgrp_fetch_session_t *session) { - rd_free(session->member_id); - RD_IF_FREE(session->partitions, rd_kafka_topic_partition_list_destroy); - rd_free(session); -} - -/** - * @brief Release all ACQUIRED records owned by \p member_id across all - * share-partition metadata in the share group. - * - * This is called when a session is closed (epoch=-1), when a session - * times out, or as part of periodic lock-expiry reclamation. - */ -void rd_kafka_mock_sgrp_release_member_locks( - rd_kafka_mock_sharegroup_t *mshgrp, - const char *member_id) { - rd_kafka_mock_sgrp_partmeta_t *pmeta; - - TAILQ_FOREACH(pmeta, &mshgrp->partitions, link) { - rd_kafka_mock_sgrp_record_state_t *state, *tmp; - TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { - if (state->state != - RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) - continue; - if (!state->owner_member_id) - continue; - if (strcmp(state->owner_member_id, member_id) != 0) - continue; - - state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE; - rd_free(state->owner_member_id); - state->owner_member_id = NULL; - state->lock_expiry_ts = 0; - } - } -} - -/** - * @brief Proactively release any expired acquisition locks. - * - * Iterates all partition metadata in the share group and flips - * ACQUIRED records whose lock_expiry_ts has passed back to AVAILABLE. - */ -static void -rd_kafka_mock_sgrp_expire_locks(rd_kafka_mock_sharegroup_t *mshgrp, - rd_ts_t now) { - rd_kafka_mock_sgrp_partmeta_t *pmeta; - - TAILQ_FOREACH(pmeta, &mshgrp->partitions, link) { - rd_kafka_mock_sgrp_record_state_t *state, *tmp; - TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { - if (state->state != - RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) - continue; - if (!state->lock_expiry_ts || - state->lock_expiry_ts > now) - continue; - - /* Lock has expired — release back to AVAILABLE */ - state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE; - RD_IF_FREE(state->owner_member_id, rd_free); - state->owner_member_id = NULL; - state->lock_expiry_ts = 0; - } - } -} - -/** - * @brief Periodic timer: expire stale share-fetch sessions and - * proactively reclaim expired acquisition locks. - */ -void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts, - void *arg) { - rd_kafka_mock_sharegroup_t *mshgrp = arg; - rd_kafka_mock_sgrp_fetch_session_t *session, *tmp; - rd_ts_t now = rd_clock(); - - (void)rkts; - - /* 1. Expire stale sessions and release their member locks. */ - TAILQ_FOREACH_SAFE(session, &mshgrp->fetch_sessions, link, tmp) { - if (session->ts_last_activity + - (mshgrp->session_timeout_ms * 1000) > - now) - continue; - - /* Release all locks held by this member before - * destroying the session. */ - rd_kafka_mock_sgrp_release_member_locks(mshgrp, - session->member_id); - - TAILQ_REMOVE(&mshgrp->fetch_sessions, session, link); - mshgrp->fetch_session_cnt--; - rd_kafka_mock_sgrp_fetch_session_destroy(session); - } - - /* 2. Proactively reclaim any expired acquisition locks. - * This catches records whose owning consumer crashed - * without closing its session cleanly. */ - rd_kafka_mock_sgrp_expire_locks(mshgrp, now); -} - /** * @brief A client connection closed, check if any cgrp has any state * for this connection that needs to be cleared. diff --git a/src/rdkafka_mock_handlers.c b/src/rdkafka_mock_handlers.c index 14c242c349..4770e69663 100644 --- a/src/rdkafka_mock_handlers.c +++ b/src/rdkafka_mock_handlers.c @@ -3998,18 +3998,18 @@ rd_kafka_mock_sgrp_write_acquired_records( const rd_kafka_mock_partition_t *mpart, const rd_kafkap_str_t *member_id, rd_ts_t now) { - const rd_kafka_mock_msgset_t *msgsets[256]; - int msgset_cnt = 0; + rd_list_t msgsets; int64_t offset; size_t total_len = 0; int i; + rd_list_init(&msgsets, 16, NULL); /* no free_cb: borrowed ptrs */ + /* Collect unique msgsets containing acquired records for this member */ for (offset = pmeta->spso; offset <= pmeta->speo; offset++) { rd_kafka_mock_sgrp_record_state_t *state = rd_kafka_mock_sgrp_record_state_find(pmeta, offset); const rd_kafka_mock_msgset_t *mset; - int j, dup; if (!state || state->state != RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) @@ -4026,26 +4026,16 @@ rd_kafka_mock_sgrp_write_acquired_records( continue; /* Deduplicate: multiple offsets may fall in the same batch */ - dup = 0; - for (j = 0; j < msgset_cnt; j++) { - if (msgsets[j] == mset) { - dup = 1; - break; - } - } - if (dup) + if (rd_list_find(&msgsets, mset, rd_list_cmp_ptr)) continue; - if (msgset_cnt >= - (int)(sizeof(msgsets) / sizeof(msgsets[0]))) - break; /* safety cap */ - - msgsets[msgset_cnt++] = mset; + rd_list_add(&msgsets, (void *)mset); total_len += RD_KAFKAP_BYTES_LEN(&mset->bytes); } - if (msgset_cnt == 0) { + if (rd_list_cnt(&msgsets) == 0) { /* No acquired records: write NULL compact bytes */ + rd_list_destroy(&msgsets); rd_kafka_buf_write_kbytes(resp, NULL); return 0; } @@ -4054,11 +4044,14 @@ rd_kafka_mock_sgrp_write_acquired_records( rd_kafka_buf_write_uvarint(resp, (uint64_t)(total_len + 1)); /* Write each msgset's raw bytes back-to-back */ - for (i = 0; i < msgset_cnt; i++) { - rd_kafka_buf_write(resp, msgsets[i]->bytes.data, - RD_KAFKAP_BYTES_LEN(&msgsets[i]->bytes)); + for (i = 0; i < rd_list_cnt(&msgsets); i++) { + const rd_kafka_mock_msgset_t *mset = + rd_list_elem(&msgsets, i); + rd_kafka_buf_write(resp, mset->bytes.data, + RD_KAFKAP_BYTES_LEN(&mset->bytes)); } + rd_list_destroy(&msgsets); return total_len; } diff --git a/src/rdkafka_mock_sharegrp.c b/src/rdkafka_mock_sharegrp.c index dc51256c36..02f71cc6a9 100644 --- a/src/rdkafka_mock_sharegrp.c +++ b/src/rdkafka_mock_sharegrp.c @@ -745,6 +745,123 @@ void rd_kafka_mock_sharegroup_set_record_lock_duration( mtx_unlock(&mcluster->lock); } +/** + * @brief Destroy share fetch session. + */ +void rd_kafka_mock_sgrp_fetch_session_destroy( + rd_kafka_mock_sgrp_fetch_session_t *session) { + rd_free(session->member_id); + RD_IF_FREE(session->partitions, rd_kafka_topic_partition_list_destroy); + rd_free(session); +} + +/** + * @brief Release all ACQUIRED records owned by \p member_id across all + * share-partition metadata in the share group. + * + * This is called when a session is closed (epoch=-1), when a session + * times out, or as part of periodic lock-expiry reclamation. + * + * @locks mcluster->lock MUST be held. + */ +void rd_kafka_mock_sgrp_release_member_locks( + rd_kafka_mock_sharegroup_t *mshgrp, + const char *member_id) { + rd_kafka_mock_sgrp_partmeta_t *pmeta; + + TAILQ_FOREACH(pmeta, &mshgrp->partitions, link) { + rd_kafka_mock_sgrp_record_state_t *state, *tmp; + TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { + if (state->state != + RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) + continue; + if (!state->owner_member_id) + continue; + if (strcmp(state->owner_member_id, member_id) != 0) + continue; + + state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE; + rd_free(state->owner_member_id); + state->owner_member_id = NULL; + state->lock_expiry_ts = 0; + } + } +} + +/** + * @brief Proactively release any expired acquisition locks. + * + * Iterates all partition metadata in the share group and flips + * ACQUIRED records whose lock_expiry_ts has passed back to AVAILABLE. + * + * @locks mcluster->lock MUST be held. + */ +static void +rd_kafka_mock_sgrp_expire_locks(rd_kafka_mock_sharegroup_t *mshgrp, + rd_ts_t now) { + rd_kafka_mock_sgrp_partmeta_t *pmeta; + + TAILQ_FOREACH(pmeta, &mshgrp->partitions, link) { + rd_kafka_mock_sgrp_record_state_t *state, *tmp; + TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { + if (state->state != + RD_KAFKA_MOCK_SGRP_RECORD_ACQUIRED) + continue; + if (!state->lock_expiry_ts || + state->lock_expiry_ts > now) + continue; + + /* Lock has expired — release back to AVAILABLE */ + state->state = RD_KAFKA_MOCK_SGRP_RECORD_AVAILABLE; + RD_IF_FREE(state->owner_member_id, rd_free); + state->owner_member_id = NULL; + state->lock_expiry_ts = 0; + } + } +} + +/** + * @brief Periodic timer: expire stale share-fetch sessions and + * proactively reclaim expired acquisition locks. + * + * @locks mcluster->lock is acquired and released. + */ +void rd_kafka_mock_sgrp_fetch_session_tmr_cb(rd_kafka_timers_t *rkts, + void *arg) { + rd_kafka_mock_sharegroup_t *mshgrp = arg; + rd_kafka_mock_sgrp_fetch_session_t *session, *tmp; + rd_ts_t now = rd_clock(); + rd_kafka_mock_cluster_t *mcluster = mshgrp->cluster; + + (void)rkts; + + mtx_lock(&mcluster->lock); + + /* 1. Expire stale sessions and release their member locks. */ + TAILQ_FOREACH_SAFE(session, &mshgrp->fetch_sessions, link, tmp) { + if (session->ts_last_activity + + (mshgrp->session_timeout_ms * 1000) > + now) + continue; + + /* Release all locks held by this member before + * destroying the session. */ + rd_kafka_mock_sgrp_release_member_locks(mshgrp, + session->member_id); + + TAILQ_REMOVE(&mshgrp->fetch_sessions, session, link); + mshgrp->fetch_session_cnt--; + rd_kafka_mock_sgrp_fetch_session_destroy(session); + } + + /* 2. Proactively reclaim any expired acquisition locks. + * This catches records whose owning consumer crashed + * without closing its session cleanly. */ + rd_kafka_mock_sgrp_expire_locks(mshgrp, now); + + mtx_unlock(&mcluster->lock); +} + /** * @brief A client connection closed, check if any sharegroup has any * state for this connection that needs to be cleared. From 01874ddb6d13133d84dea40446c9c15a11edced2 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Fri, 13 Feb 2026 16:31:56 +0530 Subject: [PATCH 18/20] Minor --- src/rdkafka_cgrp.c | 5 ----- src/rdkafka_fetcher.c | 3 --- 2 files changed, 8 deletions(-) diff --git a/src/rdkafka_cgrp.c b/src/rdkafka_cgrp.c index 4d9be006ff..062dd51cd7 100644 --- a/src/rdkafka_cgrp.c +++ b/src/rdkafka_cgrp.c @@ -6552,11 +6552,6 @@ void rd_kafka_cgrp_consumer_group_heartbeat(rd_kafka_cgrp_t *rkcg, } } - if (RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk) && - !rkcg_subscription_topics && rkcg->rkcg_subscription_topics) { - rkcg_subscription_topics = rkcg->rkcg_subscription_topics; - } - rkcg->rkcg_expedite_heartbeat_retries++; if (RD_KAFKA_IS_SHARE_CONSUMER(rkcg->rkcg_rk)) { diff --git a/src/rdkafka_fetcher.c b/src/rdkafka_fetcher.c index 3628090330..1784ac0a33 100644 --- a/src/rdkafka_fetcher.c +++ b/src/rdkafka_fetcher.c @@ -1681,9 +1681,6 @@ void rd_kafka_ShareFetchRequest( rd_kafka_buf_write_arraycnt(rkbuf, 0); } - /* Top-level tags */ - rd_kafka_buf_write_tags_empty(rkbuf); - /* Consider Fetch requests blocking if fetch.wait.max.ms >= 1s */ if (rkb->rkb_rk->rk_conf.fetch_wait_max_ms >= 1000) rkbuf->rkbuf_flags |= RD_KAFKA_OP_F_BLOCKING; From fc71c632fd2ca713e8d8f3b65bf679908d172a56 Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Fri, 13 Feb 2026 18:13:56 +0530 Subject: [PATCH 19/20] Add test to CMakeList --- ....c => 0156-kip932-sharefetch_mockbroker.c} | 591 ++++++++++-------- tests/CMakeLists.txt | 1 + tests/test.c | 2 + 3 files changed, 327 insertions(+), 267 deletions(-) rename tests/{0152-kip932-sharefetch_mockbroker.c => 0156-kip932-sharefetch_mockbroker.c} (56%) diff --git a/tests/0152-kip932-sharefetch_mockbroker.c b/tests/0156-kip932-sharefetch_mockbroker.c similarity index 56% rename from tests/0152-kip932-sharefetch_mockbroker.c rename to tests/0156-kip932-sharefetch_mockbroker.c index 1e9c07768f..72d83ecce4 100644 --- a/tests/0152-kip932-sharefetch_mockbroker.c +++ b/tests/0156-kip932-sharefetch_mockbroker.c @@ -1,39 +1,42 @@ /* - * KIP-932 ShareFetch mock broker demo using the librdkafka share consumer API. + * librdkafka - Apache Kafka C library * - * This test exercises the ShareFetch path only. There is no coordinator - * or ShareAcknowledge support in the mock broker, so group management and - * ack-based state transitions are not validated here. + * Copyright (c) 2025, Confluent Inc. + * All rights reserved. * - * Build (from repo root): - * cc -Isrc -o /tmp/kip932_share_consumer_mock \ - * tests/012x-kip932-sharefetch_mockbroker.c src/librdkafka.a -lz -lpthread + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: * - * Run: - * /tmp/kip932_share_consumer_mock + * 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. */ -#include "rdkafka.h" -#include "rdkafka_mock.h" -#include "rdkafka_protocol.h" - -#include -#include -#include -#include +#include "test.h" -static void die(const char *msg) { - fprintf(stderr, "error: %s\n", msg); - exit(1); -} +#include "../src/rdkafka_proto.h" -static void conf_set(rd_kafka_conf_t *conf, const char *name, - const char *value) { - char errstr[512]; - if (rd_kafka_conf_set(conf, name, value, errstr, sizeof(errstr)) != - RD_KAFKA_CONF_OK) - die(errstr); -} +/** + * @name KIP-932 ShareFetch mock broker tests using the share consumer API. + * + * Exercises the ShareFetch path via mock broker. There is no coordinator + * or ShareAcknowledge support in the mock broker, so group management and + * ack-based state transitions are not validated here. + */ typedef struct test_ctx_s { rd_kafka_t *producer; @@ -42,30 +45,32 @@ typedef struct test_ctx_s { } test_ctx_t; static test_ctx_t test_ctx_new(void) { - rd_kafka_conf_t *conf = rd_kafka_conf_new(); test_ctx_t ctx; + rd_kafka_conf_t *conf; + char errstr[512]; memset(&ctx, 0, sizeof(ctx)); - conf_set(conf, "test.mock.num.brokers", "3"); - ctx.producer = rd_kafka_new(RD_KAFKA_PRODUCER, conf, NULL, 0); - if (!ctx.producer) - die("Failed to create producer"); - - ctx.mcluster = rd_kafka_handle_mock_cluster(ctx.producer); - if (!ctx.mcluster) - die("Failed to get mock cluster handle"); - ctx.bootstraps = rd_kafka_mock_cluster_bootstraps(ctx.mcluster); - if (!ctx.bootstraps) - die("Failed to get mock bootstraps"); - - if (rd_kafka_mock_set_apiversion( - ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to enable ShareGroupHeartbeat"); - if (rd_kafka_mock_set_apiversion( - ctx.mcluster, RD_KAFKAP_ShareFetch, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to enable ShareFetch"); + + ctx.mcluster = test_mock_cluster_new(3, &ctx.bootstraps); + + TEST_ASSERT(rd_kafka_mock_set_apiversion(ctx.mcluster, + RD_KAFKAP_ShareGroupHeartbeat, + 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to enable ShareGroupHeartbeat"); + TEST_ASSERT( + rd_kafka_mock_set_apiversion(ctx.mcluster, RD_KAFKAP_ShareFetch, 1, + 1) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to enable ShareFetch"); + + /* Create a producer targeting the mock cluster */ + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "bootstrap.servers", ctx.bootstraps); + + ctx.producer = + rd_kafka_new(RD_KAFKA_PRODUCER, conf, errstr, sizeof(errstr)); + TEST_ASSERT(ctx.producer != NULL, "Failed to create producer: %s", + errstr); return ctx; } @@ -73,53 +78,59 @@ static test_ctx_t test_ctx_new(void) { static void test_ctx_destroy(test_ctx_t *ctx) { if (ctx->producer) rd_kafka_destroy(ctx->producer); + if (ctx->mcluster) + test_mock_cluster_destroy(ctx->mcluster); memset(ctx, 0, sizeof(*ctx)); } -static void produce_messages(rd_kafka_t *producer, const char *topic, +static void produce_messages(rd_kafka_t *producer, + const char *topic, int msgcnt) { for (int i = 0; i < msgcnt; i++) { char payload[64]; snprintf(payload, sizeof(payload), "%s-%d", topic, i); - if (rd_kafka_producev( + TEST_ASSERT( + rd_kafka_producev( producer, RD_KAFKA_V_TOPIC(topic), RD_KAFKA_V_VALUE(payload, strlen(payload)), RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY), - RD_KAFKA_V_END) != RD_KAFKA_RESP_ERR_NO_ERROR) { - die("Produce failed"); - } + RD_KAFKA_V_END) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Produce failed"); } rd_kafka_flush(producer, 5000); } static rd_kafka_share_t *new_share_consumer(const char *bootstraps, const char *group_id) { - rd_kafka_conf_t *conf = rd_kafka_conf_new(); + rd_kafka_conf_t *conf; rd_kafka_share_t *consumer; - conf_set(conf, "bootstrap.servers", bootstraps); - conf_set(conf, "group.id", group_id); + test_conf_init(&conf, NULL, 0); + test_conf_set(conf, "bootstrap.servers", bootstraps); + test_conf_set(conf, "group.id", group_id); + consumer = rd_kafka_share_consumer_new(conf, NULL, 0); - if (!consumer) - die("Failed to create consumer"); + TEST_ASSERT(consumer != NULL, "Failed to create share consumer"); rd_kafka_share_poll_set_consumer(consumer); return consumer; } static void subscribe_topics(rd_kafka_share_t *consumer, - const char **topics, int topic_cnt) { + const char **topics, + int topic_cnt) { rd_kafka_topic_partition_list_t *tpl = rd_kafka_topic_partition_list_new(topic_cnt); for (int i = 0; i < topic_cnt; i++) { - rd_kafka_topic_partition_list_add( - tpl, topics[i], RD_KAFKA_PARTITION_UA); + rd_kafka_topic_partition_list_add(tpl, topics[i], + RD_KAFKA_PARTITION_UA); } - if (rd_kafka_share_subscribe(consumer, tpl)) - die("Subscribe failed"); + TEST_ASSERT(!rd_kafka_share_subscribe(consumer, tpl), + "Subscribe failed"); rd_kafka_topic_partition_list_destroy(tpl); } -static int consume_n(rd_kafka_share_t *consumer, int expected, int max_attempts) { +static int +consume_n(rd_kafka_share_t *consumer, int expected, int max_attempts) { int consumed = 0; int attempts = 0; @@ -128,13 +139,13 @@ static int consume_n(rd_kafka_share_t *consumer, int expected, int max_attempts) size_t rcvd_msgs = 0; rd_kafka_error_t *error; - error = rd_kafka_share_consume_batch( - consumer, 500, rkmessages, &rcvd_msgs); + error = rd_kafka_share_consume_batch(consumer, 500, rkmessages, + &rcvd_msgs); attempts++; if (error) { - fprintf(stderr, "consume error: %s\n", - rd_kafka_error_string(error)); + TEST_SAY("consume error: %s\n", + rd_kafka_error_string(error)); rd_kafka_error_destroy(error); continue; } @@ -142,13 +153,13 @@ static int consume_n(rd_kafka_share_t *consumer, int expected, int max_attempts) for (size_t i = 0; i < rcvd_msgs; i++) { rd_kafka_message_t *rkmsg = rkmessages[i]; if (rkmsg->err) { - fprintf(stderr, "consume error: %s\n", - rd_kafka_message_errstr(rkmsg)); + TEST_SAY("consume error: %s\n", + rd_kafka_message_errstr(rkmsg)); rd_kafka_message_destroy(rkmsg); continue; } - printf("Consumed: %.*s\n", (int)rkmsg->len, - (const char *)rkmsg->payload); + TEST_SAY("Consumed: %.*s\n", (int)rkmsg->len, + (const char *)rkmsg->payload); consumed++; rd_kafka_message_destroy(rkmsg); } @@ -157,16 +168,19 @@ static int consume_n(rd_kafka_share_t *consumer, int expected, int max_attempts) return consumed; } -static int run_basic_consume(void) { + +static void do_test_basic_consume(void) { const char *topic = "kip932_pos_basic"; const int msgcnt = 5; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + SUB_TEST_QUICK(); + + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, msgcnt); consumer = new_share_consumer(ctx.bootstraps, "sg-pos-basic"); @@ -177,18 +191,22 @@ static int run_basic_consume(void) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == msgcnt; + TEST_ASSERT(consumed == msgcnt, + "Expected %d consumed, got %d", msgcnt, consumed); + SUB_TEST_PASS(); } -static int run_followup_fetch(void) { +static void do_test_followup_fetch(void) { const char *topic = "kip932_pos_followup"; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + SUB_TEST_QUICK(); + + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, 5); consumer = new_share_consumer(ctx.bootstraps, "sg-pos-followup"); @@ -200,19 +218,23 @@ static int run_followup_fetch(void) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == 5; + TEST_ASSERT(consumed == 5, + "Expected 5 consumed, got %d", consumed); + SUB_TEST_PASS(); } -static int run_multi_partition(void) { +static void do_test_multi_partition(void) { const char *topic = "kip932_pos_multi_part"; const int msgcnt = 6; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 2, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + SUB_TEST_QUICK(); + + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 2, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, msgcnt); consumer = new_share_consumer(ctx.bootstraps, "sg-pos-multipart"); @@ -223,23 +245,27 @@ static int run_multi_partition(void) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == msgcnt; + TEST_ASSERT(consumed == msgcnt, + "Expected %d consumed, got %d", msgcnt, consumed); + SUB_TEST_PASS(); } -static int run_multi_topic(void) { - const char *topic_a = "kip932_pos_topic_a"; - const char *topic_b = "kip932_pos_topic_b"; +static void do_test_multi_topic(void) { + const char *topic_a = "kip932_pos_topic_a"; + const char *topic_b = "kip932_pos_topic_b"; const char *topics[] = {topic_a, topic_b}; - test_ctx_t ctx = test_ctx_new(); + test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic_a, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic A"); - if (rd_kafka_mock_topic_create(ctx.mcluster, topic_b, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic B"); + SUB_TEST_QUICK(); + + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic_a, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic A"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic_b, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic B"); produce_messages(ctx.producer, topic_a, 2); produce_messages(ctx.producer, topic_b, 2); @@ -251,18 +277,22 @@ static int run_multi_topic(void) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == 4; + TEST_ASSERT(consumed == 4, + "Expected 4 consumed, got %d", consumed); + SUB_TEST_PASS(); } -static int run_empty_topic_no_records(void) { +static void do_test_empty_topic_no_records(void) { const char *topic = "kip932_pos_empty"; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + SUB_TEST_QUICK(); + + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); consumer = new_share_consumer(ctx.bootstraps, "sg-pos-empty"); subscribe_topics(consumer, &topic, 1); @@ -272,22 +302,24 @@ static int run_empty_topic_no_records(void) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == 0; + TEST_ASSERT(consumed == 0, + "Expected 0 consumed, got %d", consumed); + SUB_TEST_PASS(); } -static int run_negative_sharefetch_error(rd_kafka_resp_err_t err) { +static void do_test_negative_sharefetch_error(rd_kafka_resp_err_t err) { const char *topic = "kip932_neg_sharefetch_error"; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, 1); - rd_kafka_mock_push_request_errors(ctx.mcluster, RD_KAFKAP_ShareFetch, - 1, err); + rd_kafka_mock_push_request_errors(ctx.mcluster, RD_KAFKAP_ShareFetch, 1, + err); consumer = new_share_consumer(ctx.bootstraps, "sg-neg-sharefetch"); subscribe_topics(consumer, &topic, 1); @@ -297,33 +329,37 @@ static int run_negative_sharefetch_error(rd_kafka_resp_err_t err) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == 0; + TEST_ASSERT(consumed == 0, + "Expected 0 consumed, got %d", consumed); } -static int run_sharefetch_invalid_session_epoch(void) { - return run_negative_sharefetch_error( +static void do_test_sharefetch_invalid_session_epoch(void) { + SUB_TEST_QUICK(); + do_test_negative_sharefetch_error( RD_KAFKA_RESP_ERR_INVALID_FETCH_SESSION_EPOCH); + SUB_TEST_PASS(); } -static int run_sharefetch_unknown_topic_or_part(void) { - return run_negative_sharefetch_error( +static void do_test_sharefetch_unknown_topic_or_part(void) { + SUB_TEST_QUICK(); + do_test_negative_sharefetch_error( RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART); + SUB_TEST_PASS(); } -static int run_sghb_error(rd_kafka_resp_err_t err, int count) { +static void do_test_sghb_error(rd_kafka_resp_err_t err, int count) { const char *topic = "kip932_neg_sghb"; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, 1); - rd_kafka_mock_push_request_errors(ctx.mcluster, - RD_KAFKAP_ShareGroupHeartbeat, count, - err); + rd_kafka_mock_push_request_errors( + ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, count, err); consumer = new_share_consumer(ctx.bootstraps, "sg-neg-sghb"); subscribe_topics(consumer, &topic, 1); @@ -333,23 +369,25 @@ static int run_sghb_error(rd_kafka_resp_err_t err, int count) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == 0; + TEST_ASSERT(consumed == 0, + "Expected 0 consumed, got %d", consumed); } -static int run_sghb_coord_unavailable(void) { - return run_sghb_error( - RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, 50); +static void do_test_sghb_coord_unavailable(void) { + SUB_TEST_QUICK(); + do_test_sghb_error(RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, 50); + SUB_TEST_PASS(); } -static int run_topic_error(rd_kafka_resp_err_t err) { +static void do_test_topic_error(rd_kafka_resp_err_t err) { const char *topic = "kip932_neg_topic_error"; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, 1); rd_kafka_mock_topic_set_error(ctx.mcluster, topic, err); @@ -361,20 +399,24 @@ static int run_topic_error(rd_kafka_resp_err_t err) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == 0; + TEST_ASSERT(consumed == 0, + "Expected 0 consumed, got %d", consumed); } -static int run_topic_error_unknown_topic_or_part(void) { - return run_topic_error( - RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART); +static void do_test_topic_error_unknown_topic_or_part(void) { + SUB_TEST_QUICK(); + do_test_topic_error(RD_KAFKA_RESP_ERR_UNKNOWN_TOPIC_OR_PART); + SUB_TEST_PASS(); } -static int run_unknown_topic_subscription(void) { +static void do_test_unknown_topic_subscription(void) { const char *topic = "kip932_neg_unknown_topic"; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; + SUB_TEST_QUICK(); + consumer = new_share_consumer(ctx.bootstraps, "sg-neg-unknown-topic"); subscribe_topics(consumer, &topic, 1); consumed = consume_n(consumer, 1, 5); @@ -383,18 +425,22 @@ static int run_unknown_topic_subscription(void) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == 0; + TEST_ASSERT(consumed == 0, + "Expected 0 consumed, got %d", consumed); + SUB_TEST_PASS(); } -static int run_empty_fetch_no_records(void) { +static void do_test_empty_fetch_no_records(void) { const char *topic = "kip932_neg_empty_fetch"; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + SUB_TEST_QUICK(); + + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); consumer = new_share_consumer(ctx.bootstraps, "sg-neg-empty"); subscribe_topics(consumer, &topic, 1); @@ -404,7 +450,9 @@ static int run_empty_fetch_no_records(void) { rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == 0; + TEST_ASSERT(consumed == 0, + "Expected 0 consumed, got %d", consumed); + SUB_TEST_PASS(); } /** @@ -412,40 +460,39 @@ static int run_empty_fetch_no_records(void) { * (UNKNOWN_MEMBER_ID), and that after the member re-joins it can * consume again. * - * Phase 1: Consumer joins normally via SGHB → consumes messages OK. - * Phase 2: Push SGHB errors → heartbeats fail → member expires → broker + * Phase 1: Consumer joins normally via SGHB -> consumes messages OK. + * Phase 2: Push SGHB errors -> heartbeats fail -> member expires -> broker * rejects ShareFetch with UNKNOWN_MEMBER_ID. - * Phase 3: SGHB errors drain → member re-joins → consumes again. + * Phase 3: SGHB errors drain -> member re-joins -> consumes again. */ -static int run_member_validation(void) { +static void do_test_member_validation(void) { const char *topic = "kip932_member_validation"; const int msgcnt = 4; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed_p1, consumed_p3; + SUB_TEST(); + /* Short session timeout so the member is evicted quickly once * heartbeats stop succeeding. */ rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 500); - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, msgcnt); consumer = new_share_consumer(ctx.bootstraps, "sg-member-val"); subscribe_topics(consumer, &topic, 1); - /* Phase 1: Consume normally — member is registered via SGHB. */ + /* Phase 1: Consume normally -- member is registered via SGHB. */ consumed_p1 = consume_n(consumer, 2, 30); - printf(" member_validation: phase1 consumed %d/2\n", consumed_p1); + TEST_SAY("member_validation: phase1 consumed %d/2\n", consumed_p1); /* Phase 2: Block SGHB so heartbeats fail. * Push enough errors to cover the window while we wait for the - * member to be evicted. The client sends SGHB roughly every - * heartbeat interval (~5s default), but with a 500ms session - * timeout the member will be removed well before the errors - * drain. We push 20 errors to be safe. */ + * member to be evicted. */ rd_kafka_mock_push_request_errors( ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, 20, RD_KAFKA_RESP_ERR_COORDINATOR_NOT_AVAILABLE, @@ -476,28 +523,33 @@ static int run_member_validation(void) { * succeeds, the member re-joins and the remaining records * become fetchable again. */ consumed_p3 = consume_n(consumer, 2, 50); - printf(" member_validation: phase3 consumed %d/2\n", consumed_p3); + TEST_SAY("member_validation: phase3 consumed %d/2\n", consumed_p3); rd_kafka_share_consumer_close(consumer); rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed_p1 >= 2 && (consumed_p1 + consumed_p3) >= msgcnt; + TEST_ASSERT(consumed_p1 >= 2 && (consumed_p1 + consumed_p3) >= msgcnt, + "Expected at least 2+2, got %d+%d", + consumed_p1, consumed_p3); + SUB_TEST_PASS(); } -static int run_sharefetch_session_expiry_rtt(void) { +static void do_test_sharefetch_session_expiry_rtt(void) { const char *topic = "kip932_rtt_expiry"; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; + SUB_TEST(); + /* Session timeout must be long enough for normal requests * to complete, but short enough to expire during high RTT. */ rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 1000); - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, 2); consumer = new_share_consumer(ctx.bootstraps, "sg-rtt-expiry"); @@ -505,7 +557,7 @@ static int run_sharefetch_session_expiry_rtt(void) { /* Phase 1: consume one message with normal RTT (no injection). */ consumed = consume_n(consumer, 1, 20); - printf(" rtt_expiry: phase1 consumed %d/1\n", consumed); + TEST_SAY("rtt_expiry: phase1 consumed %d/1\n", consumed); /* Phase 2: inject RTT >> session timeout to force session expiry. * All requests to broker 1 now take 3s, but the session @@ -513,21 +565,21 @@ static int run_sharefetch_session_expiry_rtt(void) { rd_kafka_mock_broker_set_rtt(ctx.mcluster, 1, 3000); usleep(2000 * 1000); /* wait for session to expire */ - /* Phase 3: clear RTT and let the consumer recover. - * It should re-create the session (epoch=0) and consume - * the remaining message. */ + /* Phase 3: clear RTT and let the consumer recover. */ rd_kafka_mock_broker_set_rtt(ctx.mcluster, 1, 0); consumed += consume_n(consumer, 1, 30); - printf(" rtt_expiry: phase3 consumed %d/2 total\n", consumed); + TEST_SAY("rtt_expiry: phase3 consumed %d/2 total\n", consumed); rd_kafka_share_consumer_close(consumer); rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == 2; + TEST_ASSERT(consumed == 2, + "Expected 2 consumed, got %d", consumed); + SUB_TEST_PASS(); } -static int run_forgotten_topics(void) { +static void do_test_forgotten_topics(void) { const char *topic_a = "kip932_forgotten_a"; const char *topic_b = "kip932_forgotten_b"; const char *both[] = {topic_a, topic_b}; @@ -535,12 +587,14 @@ static int run_forgotten_topics(void) { rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic_a, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic A"); - if (rd_kafka_mock_topic_create(ctx.mcluster, topic_b, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic B"); + SUB_TEST_QUICK(); + + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic_a, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic A"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic_b, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic B"); /* Produce 2 messages to each topic */ produce_messages(ctx.producer, topic_a, 2); @@ -550,8 +604,8 @@ static int run_forgotten_topics(void) { consumer = new_share_consumer(ctx.bootstraps, "sg-forgotten"); subscribe_topics(consumer, both, 2); consumed = consume_n(consumer, 4, 40); - printf(" forgotten_topics: consumed %d/4 from both topics\n", - consumed); + TEST_SAY("forgotten_topics: consumed %d/4 from both topics\n", + consumed); /* Re-subscribe to only topic_a (topic_b becomes forgotten) */ subscribe_topics(consumer, &topic_a, 1); @@ -559,10 +613,10 @@ static int run_forgotten_topics(void) { /* Produce 2 more messages to topic_a */ produce_messages(ctx.producer, topic_a, 2); - /* Consume the 2 new messages — only topic_a should deliver */ + /* Consume the 2 new messages -- only topic_a should deliver */ consumed += consume_n(consumer, 2, 30); - printf(" forgotten_topics: consumed %d/6 total after forget\n", - consumed); + TEST_SAY("forgotten_topics: consumed %d/6 total after forget\n", + consumed); rd_kafka_share_consumer_close(consumer); rd_kafka_share_destroy(consumer); @@ -571,89 +625,91 @@ static int run_forgotten_topics(void) { /* We expect at least the 4 initial + 2 from topic_a = 6. * Depending on timing the consumer may or may not have already * received all messages from the first round, so we accept >= 4. */ - return consumed >= 4; + TEST_ASSERT(consumed >= 4, + "Expected at least 4 consumed, got %d", consumed); + SUB_TEST_PASS(); } -/** - * @brief Multi-consumer lock expiry test. - * - * Consumer A acquires records, then crashes (destroyed without close). - * After the lock expiry timeout, consumer B should be able to pick up - * the same records because the proactive lock-expiry scan releases them. - */ /** * @brief Produce messages one-at-a-time (each flush creates a separate * msgset on the mock partition), then consume and verify all are - * received. This validates that the ShareFetch response includes + * received. This validates that the ShareFetch response includes * records from *all* acquired msgsets, not just the first one. */ -static int run_multi_batch_consume(void) { +static void do_test_multi_batch_consume(void) { const char *topic = "kip932_multi_batch"; const int msgcnt = 5; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + SUB_TEST_QUICK(); + + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); /* Produce each message individually with a flush in between, * guaranteeing separate msgsets on the mock partition. */ for (int i = 0; i < msgcnt; i++) { char payload[64]; snprintf(payload, sizeof(payload), "batch-%d", i); - if (rd_kafka_producev( + TEST_ASSERT( + rd_kafka_producev( ctx.producer, RD_KAFKA_V_TOPIC(topic), RD_KAFKA_V_VALUE(payload, strlen(payload)), RD_KAFKA_V_MSGFLAGS(RD_KAFKA_MSG_F_COPY), - RD_KAFKA_V_END) != RD_KAFKA_RESP_ERR_NO_ERROR) - die("Produce failed"); + RD_KAFKA_V_END) == RD_KAFKA_RESP_ERR_NO_ERROR, + "Produce failed"); rd_kafka_flush(ctx.producer, 5000); } consumer = new_share_consumer(ctx.bootstraps, "sg-multi-batch"); subscribe_topics(consumer, &topic, 1); consumed = consume_n(consumer, msgcnt, 50); - printf(" multi_batch: consumed %d/%d\n", consumed, msgcnt); + TEST_SAY("multi_batch: consumed %d/%d\n", consumed, msgcnt); rd_kafka_share_consumer_close(consumer); rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed == msgcnt; + TEST_ASSERT(consumed == msgcnt, + "Expected %d consumed, got %d", msgcnt, consumed); + SUB_TEST_PASS(); } /** * @brief Verify that max_delivery_attempts causes records to be archived - * after the limit is exceeded. Consumer A acquires all records, then - * its session times out (releasing locks). Consumer B acquires them - * again, and its session also times out. After the delivery limit is + * after the limit is exceeded. Consumer A acquires all records, then + * its session times out (releasing locks). Consumer B acquires them + * again, and its session also times out. After the delivery limit is * exhausted, Consumer C should see 0 available records. */ -static int run_max_delivery_attempts(void) { +static void do_test_max_delivery_attempts(void) { const char *topic = "kip932_max_delivery"; const int msgcnt = 3; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed_a, consumed_b, consumed_c; + SUB_TEST(); + /* Set max delivery attempts to 2 and a short session timeout * so locks expire quickly after consumer destruction. */ rd_kafka_mock_sharegroup_set_max_delivery_attempts(ctx.mcluster, 2); rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 500); - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, msgcnt); /* Delivery 1: Consumer A acquires and "crashes" (no ack). */ consumer = new_share_consumer(ctx.bootstraps, "sg-max-delivery"); subscribe_topics(consumer, &topic, 1); consumed_a = consume_n(consumer, msgcnt, 50); - printf(" max_delivery: A consumed %d/%d (delivery 1)\n", - consumed_a, msgcnt); + TEST_SAY("max_delivery: A consumed %d/%d (delivery 1)\n", + consumed_a, msgcnt); rd_kafka_share_destroy(consumer); usleep(1500 * 1000); /* wait for lock expiry */ @@ -662,8 +718,8 @@ static int run_max_delivery_attempts(void) { consumer = new_share_consumer(ctx.bootstraps, "sg-max-delivery"); subscribe_topics(consumer, &topic, 1); consumed_b = consume_n(consumer, msgcnt, 50); - printf(" max_delivery: B consumed %d/%d (delivery 2)\n", - consumed_b, msgcnt); + TEST_SAY("max_delivery: B consumed %d/%d (delivery 2)\n", + consumed_b, msgcnt); rd_kafka_share_destroy(consumer); usleep(1500 * 1000); /* wait for lock expiry */ @@ -672,45 +728,51 @@ static int run_max_delivery_attempts(void) { consumer = new_share_consumer(ctx.bootstraps, "sg-max-delivery"); subscribe_topics(consumer, &topic, 1); consumed_c = consume_n(consumer, 1, 10); - printf(" max_delivery: C consumed %d/0 (should be archived)\n", - consumed_c); + TEST_SAY("max_delivery: C consumed %d/0 (should be archived)\n", + consumed_c); rd_kafka_share_consumer_close(consumer); rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed_a == msgcnt && consumed_b == msgcnt && consumed_c == 0; + TEST_ASSERT(consumed_a == msgcnt && consumed_b == msgcnt && + consumed_c == 0, + "Expected A=%d B=%d C=0, got A=%d B=%d C=%d", + msgcnt, msgcnt, consumed_a, consumed_b, consumed_c); + SUB_TEST_PASS(); } /** * @brief Verify that record_lock_duration_ms controls how long acquired * records stay locked, independently of session_timeout_ms. * Sets a short lock duration (300ms) with a longer session timeout - * (10s). Consumer A acquires records and "crashes". After the short + * (10s). Consumer A acquires records and "crashes". After the short * lock duration expires, Consumer B should be able to acquire them * even though A's session hasn't timed out yet. */ -static int run_record_lock_duration(void) { +static void do_test_record_lock_duration(void) { const char *topic = "kip932_lock_duration"; const int msgcnt = 3; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed_a, consumed_b; + SUB_TEST(); + /* Long session timeout, short record lock duration. */ rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 10000); rd_kafka_mock_sharegroup_set_record_lock_duration(ctx.mcluster, 300); - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, msgcnt); /* Consumer A acquires records, then crashes (no close). */ consumer = new_share_consumer(ctx.bootstraps, "sg-lock-duration"); subscribe_topics(consumer, &topic, 1); consumed_a = consume_n(consumer, msgcnt, 50); - printf(" lock_duration: A consumed %d/%d\n", consumed_a, msgcnt); + TEST_SAY("lock_duration: A consumed %d/%d\n", consumed_a, msgcnt); rd_kafka_share_destroy(consumer); /* Wait for record lock to expire (300ms + margin), @@ -722,28 +784,40 @@ static int run_record_lock_duration(void) { consumer = new_share_consumer(ctx.bootstraps, "sg-lock-duration"); subscribe_topics(consumer, &topic, 1); consumed_b = consume_n(consumer, msgcnt, 50); - printf(" lock_duration: B consumed %d/%d\n", consumed_b, msgcnt); + TEST_SAY("lock_duration: B consumed %d/%d\n", consumed_b, msgcnt); rd_kafka_share_consumer_close(consumer); rd_kafka_share_destroy(consumer); test_ctx_destroy(&ctx); - return consumed_a == msgcnt && consumed_b == msgcnt; + TEST_ASSERT(consumed_a == msgcnt && consumed_b == msgcnt, + "Expected A=%d B=%d, got A=%d B=%d", + msgcnt, msgcnt, consumed_a, consumed_b); + SUB_TEST_PASS(); } -static int run_multi_consumer_lock_expiry(void) { +/** + * @brief Multi-consumer lock expiry test. + * + * Consumer A acquires records, then crashes (destroyed without close). + * After the lock expiry timeout, consumer B should be able to pick up + * the same records because the proactive lock-expiry scan releases them. + */ +static void do_test_multi_consumer_lock_expiry(void) { const char *topic = "kip932_multi_consumer_lock"; const int msgcnt = 3; test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer_a, *consumer_b; int consumed_a, consumed_b; + SUB_TEST(); + /* Use a short session/lock timeout so the test runs quickly. */ rd_kafka_mock_sharegroup_set_session_timeout(ctx.mcluster, 500); - if (rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) != - RD_KAFKA_RESP_ERR_NO_ERROR) - die("Failed to create mock topic"); + TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == + RD_KAFKA_RESP_ERR_NO_ERROR, + "Failed to create mock topic"); produce_messages(ctx.producer, topic, msgcnt); /* Consumer A: subscribe and consume all records (acquires locks). */ @@ -751,7 +825,7 @@ static int run_multi_consumer_lock_expiry(void) { new_share_consumer(ctx.bootstraps, "sg-multi-consumer-lock"); subscribe_topics(consumer_a, &topic, 1); consumed_a = consume_n(consumer_a, msgcnt, 50); - printf(" multi_consumer: A consumed %d/%d\n", consumed_a, msgcnt); + TEST_SAY("multi_consumer: A consumed %d/%d\n", consumed_a, msgcnt); /* Simulate crash: destroy consumer A without calling close. * The session will time out and the proactive lock-expiry @@ -767,60 +841,43 @@ static int run_multi_consumer_lock_expiry(void) { new_share_consumer(ctx.bootstraps, "sg-multi-consumer-lock"); subscribe_topics(consumer_b, &topic, 1); consumed_b = consume_n(consumer_b, msgcnt, 50); - printf(" multi_consumer: B consumed %d/%d\n", consumed_b, msgcnt); + TEST_SAY("multi_consumer: B consumed %d/%d\n", consumed_b, msgcnt); rd_kafka_share_consumer_close(consumer_b); rd_kafka_share_destroy(consumer_b); test_ctx_destroy(&ctx); - return consumed_a == msgcnt && consumed_b == msgcnt; + TEST_ASSERT(consumed_a == msgcnt && consumed_b == msgcnt, + "Expected A=%d B=%d, got A=%d B=%d", + msgcnt, msgcnt, consumed_a, consumed_b); + SUB_TEST_PASS(); } -static int run_test(const char *name, int (*fn)(void)) { - int ok = fn(); - printf("[%s] %s\n", ok ? "OK" : "FAIL", name); - return ok ? 0 : 1; -} - -int main(void) { - int failures = 0; - printf("ShareFetch test scenarios\n"); +int main_0156_kip932_sharefetch_mockbroker(int argc, char **argv) { + TEST_SKIP_MOCK_CLUSTER(0); /* Positive scenarios */ - failures += run_test("basic_consume", run_basic_consume); - failures += run_test("followup_fetch", run_followup_fetch); - failures += run_test("multi_partition", run_multi_partition); - failures += run_test("multi_topic", run_multi_topic); - failures += run_test("empty_topic_no_records", run_empty_topic_no_records); - failures += run_test("sharefetch_session_expiry_rtt", - run_sharefetch_session_expiry_rtt); - failures += run_test("forgotten_topics", run_forgotten_topics); - failures += run_test("multi_batch_consume", run_multi_batch_consume); - failures += run_test("max_delivery_attempts", - run_max_delivery_attempts); - failures += run_test("record_lock_duration", - run_record_lock_duration); - failures += run_test("multi_consumer_lock_expiry", - run_multi_consumer_lock_expiry); + do_test_basic_consume(); + do_test_followup_fetch(); + do_test_multi_partition(); + do_test_multi_topic(); + do_test_empty_topic_no_records(); + do_test_sharefetch_session_expiry_rtt(); + do_test_forgotten_topics(); + do_test_multi_batch_consume(); + do_test_max_delivery_attempts(); + do_test_record_lock_duration(); + do_test_multi_consumer_lock_expiry(); /* Negative scenarios */ - failures += run_test("sharefetch_invalid_session_epoch", - run_sharefetch_invalid_session_epoch); - failures += run_test("sharefetch_unknown_topic_or_part", - run_sharefetch_unknown_topic_or_part); - failures += run_test("sghb_coord_unavailable", - run_sghb_coord_unavailable); - failures += run_test("topic_error_unknown_topic_or_part", - run_topic_error_unknown_topic_or_part); - failures += run_test("unknown_topic_subscription", - run_unknown_topic_subscription); - failures += run_test("empty_fetch_no_records", - run_empty_fetch_no_records); - failures += run_test("member_validation", - run_member_validation); - - printf("Failures: %d\n", failures); - return failures ? 1 : 0; + do_test_sharefetch_invalid_session_epoch(); + do_test_sharefetch_unknown_topic_or_part(); + do_test_sghb_coord_unavailable(); + do_test_topic_error_unknown_topic_or_part(); + do_test_unknown_topic_subscription(); + do_test_empty_fetch_no_records(); + do_test_member_validation(); + + return 0; } - diff --git a/tests/CMakeLists.txt b/tests/CMakeLists.txt index 243309dfd6..88fa3eee3d 100644 --- a/tests/CMakeLists.txt +++ b/tests/CMakeLists.txt @@ -146,6 +146,7 @@ set( 0153-memberid.c 0154-share-consumer.c 0155-share_group_heartbeat_mock.c + 0156-kip932-sharefetch_mockbroker.c 8000-idle.cpp 8001-fetch_from_follower_mock_manual.c test.c diff --git a/tests/test.c b/tests/test.c index fda75fac32..300cbeb079 100644 --- a/tests/test.c +++ b/tests/test.c @@ -274,6 +274,7 @@ _TEST_DECL(0152_rebootstrap_local); _TEST_DECL(0153_memberid); _TEST_DECL(0154_share_consumer); _TEST_DECL(0155_share_group_heartbeat_mock); +_TEST_DECL(0156_kip932_sharefetch_mockbroker); /* Manual tests */ _TEST_DECL(8000_idle); @@ -544,6 +545,7 @@ struct test tests[] = { _TEST(0153_memberid, TEST_F_LOCAL), _TEST(0154_share_consumer, 0, TEST_BRKVER(0, 4, 0, 0)), _TEST(0155_share_group_heartbeat_mock, TEST_F_LOCAL), + _TEST(0156_kip932_sharefetch_mockbroker, TEST_F_LOCAL), /* Manual tests */ _TEST(8000_idle, TEST_F_MANUAL), From a6c56c6218b8ecf0f61ca3927a51b0d9cce42a3e Mon Sep 17 00:00:00 2001 From: Kaushik Raina Date: Fri, 13 Feb 2026 18:25:14 +0530 Subject: [PATCH 20/20] Fix minor test failure --- tests/0156-kip932-sharefetch_mockbroker.c | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/0156-kip932-sharefetch_mockbroker.c b/tests/0156-kip932-sharefetch_mockbroker.c index 72d83ecce4..0b375d762e 100644 --- a/tests/0156-kip932-sharefetch_mockbroker.c +++ b/tests/0156-kip932-sharefetch_mockbroker.c @@ -352,14 +352,24 @@ static void do_test_sghb_error(rd_kafka_resp_err_t err, int count) { test_ctx_t ctx = test_ctx_new(); rd_kafka_share_t *consumer; int consumed; + rd_kafka_resp_err_t *errs; + int i; TEST_ASSERT(rd_kafka_mock_topic_create(ctx.mcluster, topic, 1, 1) == RD_KAFKA_RESP_ERR_NO_ERROR, "Failed to create mock topic"); produce_messages(ctx.producer, topic, 1); - rd_kafka_mock_push_request_errors( - ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, count, err); + /* Build an array of 'count' identical errors and push them all. + * Using the array variant avoids UB from mismatched varargs count. */ + errs = malloc(sizeof(*errs) * count); + TEST_ASSERT(errs != NULL, "malloc failed"); + for (i = 0; i < count; i++) + errs[i] = err; + rd_kafka_mock_push_request_errors_array( + ctx.mcluster, RD_KAFKAP_ShareGroupHeartbeat, + (size_t)count, errs); + free(errs); consumer = new_share_consumer(ctx.bootstraps, "sg-neg-sghb"); subscribe_topics(consumer, &topic, 1);