MockBroker: Implement ShareFetch API#5302
MockBroker: Implement ShareFetch API#5302Ankith L (Ankith-Confluent) merged 21 commits intodev_kip-932_queues-for-kafkafrom
Conversation
|
🎉 All Contributor License Agreements have been signed. Ready to merge. |
src/rdkafka_fetcher.c
Outdated
|
|
||
| ApiVersion = rd_kafka_broker_ApiVersion_supported(rkb, RD_KAFKAP_ShareFetch, | ||
| 1, 1, NULL); | ||
| 0, 0, NULL); |
There was a problem hiding this comment.
Java version of Share Fetch supports version (1, 1) and above, so I think this should also be the same
src/rdkafka_mock.c
Outdated
| rd_kafka_mock_cgrp_classic_t *mcgrp_classic; | ||
| rd_kafka_mock_cgrp_consumer_t *mcgrp_consumer; | ||
| rd_kafka_mock_sharegroup_t *mshgrp; | ||
| rd_kafka_mock_sgrp_t *msgrp; |
There was a problem hiding this comment.
This is actually duplicate.
rd_kafka_mock_sharegroup_t already exists
src/rdkafka_mock.c
Outdated
| while ((mshgrp = TAILQ_FIRST(&mcluster->sharegrps))) | ||
| rd_kafka_mock_sharegroup_destroy(mshgrp); | ||
|
|
||
| while ((msgrp = TAILQ_FIRST(&mcluster->sgrps_share))) |
There was a problem hiding this comment.
same regarding this, this is also duplicate.
src/rdkafka_mock.c
Outdated
| TAILQ_INIT(&mcluster->cgrps_consumer); | ||
|
|
||
| rd_kafka_mock_sharegrps_init(mcluster); | ||
| TAILQ_INIT(&mcluster->sgrps_share); |
There was a problem hiding this comment.
The init is already present
rd_kafka_mock_sharegrps_init(mcluster);
src/rdkafka_mock_cgrp.c
Outdated
| * @locks mcluster->lock MUST be held. | ||
| */ | ||
| rd_kafka_mock_sgrp_t * | ||
| rd_kafka_mock_sgrp_find(rd_kafka_mock_cluster_t *mcluster, |
There was a problem hiding this comment.
This is already present in src/rdkafka_mock_sharegrp.c
src/rdkafka_mock_int.h
Outdated
| 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, |
There was a problem hiding this comment.
find, get, and destroy are duplicates
src/rdkafka_mock_handlers.c
Outdated
| } | ||
|
|
||
| rd_kafka_buf_skip_tags(rkbuf); | ||
| /* ForgottenToppars */ |
There was a problem hiding this comment.
I didnt understand why this code block is present in the metadata_handler.
Shouldnt this be part of share fetch handler
src/rdkafka_mock_handlers.c
Outdated
| TAILQ_FOREACH_SAFE(state, &pmeta->inflight, link, tmp) { | ||
| if (state->state != RD_KAFKA_MOCK_SGRP_RECORD_ARCHIVED) | ||
| continue; | ||
| if (state->offset <= pmeta->speo) |
There was a problem hiding this comment.
The messages will not be pruned in every scenario, because the offset can never be more than speo.
So it should be state->offset >= pmeta->spso instead.
| rd_kafka_buf_write_kbytes( | ||
| resp, mset ? &mset->bytes : NULL); | ||
| /* Response: AcquiredRecords */ | ||
| rd_kafka_buf_write_arraycnt(resp, 0); |
There was a problem hiding this comment.
Is there a reason for it to be 0?
There was a problem hiding this comment.
Yes, we have kept acknolegement workflow seperate
src/rdkafka_mock_handlers.c
Outdated
| } | ||
|
|
||
| static const rd_kafka_mock_msgset_t * | ||
| rd_kafka_mock_sgrp_first_acquired_msgset( |
There was a problem hiding this comment.
why are we sending only the first message set?
Shouldnt we send all the sets in the acquired offset range
src/rdkafka_mock_cgrp.c
Outdated
| rd_free(mcgrp); | ||
| } | ||
|
|
||
| /** |
There was a problem hiding this comment.
can we shift these functions to rdkafka_mock_sharegrp.c
src/rdkafka_mock_cgrp.c
Outdated
| * @brief Destroy share-partition metadata. | ||
| */ | ||
| static void | ||
| rd_kafka_mock_sgrp_partmeta_destroy(rd_kafka_mock_sgrp_partmeta_t *pmeta) { |
There was a problem hiding this comment.
This function is not being used anywhere
what is the use of this?
Actually sharegroup_destroy() handles it, so this is not required I guess
src/rdkafka_mock_cgrp.c
Outdated
| session->member_id); | ||
|
|
||
| TAILQ_REMOVE(&mshgrp->fetch_sessions, session, link); | ||
| mshgrp->fetch_session_cnt--; |
There was a problem hiding this comment.
I think this requires lock to modify share group related data?
src/rdkafka_mock_handlers.c
Outdated
| 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]; |
There was a problem hiding this comment.
is 256 intentional?
Ankith L (Ankith-Confluent)
left a comment
There was a problem hiding this comment.
Hey Kaushik Raina (@k-raina)
Thanks for the PR!
LGTM!
- 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.
6f6aba9 to
01874dd
Compare
…to kraina-mock-broker-share-fetch
cac0c73
into
dev_kip-932_queues-for-kafka
* Mock Broker : Implement share fetch * ShareGroupHeartBeat MVP * 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. * Add function to retrieve member IDs from a sharegroup and update assignment handling * Mock Broker : Implement share fetch * Fix SGHB return and fetcher api version * Add test for share fetch API * Fix session workflow * Update API version * Add handing for ForgottenTopicsData * Add lock expiry logic * Remove duplicates with sghb * Address feedbacks * Implement max delivery attempts and record lock * Add member validation for share fetch * Remove tmp produce code * Implement feedback * Minor * Add test to CMakeList * Fix minor test failure --------- Co-authored-by: Ankith-Confluent <[email protected]>
* Mock Broker : Implement share fetch * ShareGroupHeartBeat MVP * 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. * Add function to retrieve member IDs from a sharegroup and update assignment handling * Mock Broker : Implement share fetch * Fix SGHB return and fetcher api version * Add test for share fetch API * Fix session workflow * Update API version * Add handing for ForgottenTopicsData * Add lock expiry logic * Remove duplicates with sghb * Address feedbacks * Implement max delivery attempts and record lock * Add member validation for share fetch * Remove tmp produce code * Implement feedback * Minor * Add test to CMakeList * Fix minor test failure --------- Co-authored-by: Ankith-Confluent <[email protected]>
* Mock Broker : Implement share fetch * ShareGroupHeartBeat MVP * 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. * Add function to retrieve member IDs from a sharegroup and update assignment handling * Mock Broker : Implement share fetch * Fix SGHB return and fetcher api version * Add test for share fetch API * Fix session workflow * Update API version * Add handing for ForgottenTopicsData * Add lock expiry logic * Remove duplicates with sghb * Address feedbacks * Implement max delivery attempts and record lock * Add member validation for share fetch * Remove tmp produce code * Implement feedback * Minor * Add test to CMakeList * Fix minor test failure --------- Co-authored-by: Ankith-Confluent <[email protected]>
Summary
Implemented KIP‑932 ShareFetch in the mock broker, including session management, partition metadata (SPSO/SPEO), inflight record state, acquisition logic, and response construction.
Wired ShareFetch/ShareGroupHeartbeat ApiVersions, fixed client ShareFetch request versioning, and ensured SGHB sends subscription topics.
Added a comprehensive ShareFetch mock‑broker test suite (share consumer) with positive/negative scenarios and RTT‑expiry coverage.
ShareFetch Workflow
High level Classe and Data structures