-
Notifications
You must be signed in to change notification settings - Fork 3.2k
[KIP-848] Use metadata cache by topic id, fixes and improvements #4661
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
emasab
commented
Mar 26, 2024
- use metadata cache by topic id
- rename 'generic' protocol to 'classic'
- consumer group serve timer to awake the loop earlier
- compare and find into topic partition list by topic id only
- fix memory leak when instance creation fails and app_conf is provided
- fix cases where HB response is received after unsubscription
- use topic name from current assignment if it's missing from metadata
b36ead0
to
268cb01
Compare
6667956
to
ca1eb2d
Compare
115eba5
to
d04ed7e
Compare
ca1eb2d
to
953f668
Compare
d04ed7e
to
f05bc17
Compare
953f668
to
e4614be
Compare
f05bc17
to
bc427e2
Compare
e4614be
to
10e7cc1
Compare
bc427e2
to
117d317
Compare
10e7cc1
to
5789307
Compare
117d317
to
ba3ee8d
Compare
5789307
to
434da75
Compare
ba3ee8d
to
f405243
Compare
434da75
to
e5f3101
Compare
f5ffbf8
to
7403c10
Compare
e5f3101
to
109d348
Compare
2b56329
to
6a8f931
Compare
109d348
to
3dd0e45
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Checked all files except rdkafka_cgrp.c
. Have some doubts in that so we can discuss them first.
src/rdkafka_conf.c
Outdated
@@ -1132,19 +1132,19 @@ static const struct rd_kafka_property rd_kafka_properties[] = { | |||
"Group session keepalive heartbeat interval.", 1, 3600 * 1000, 3 * 1000}, | |||
{_RK_GLOBAL | _RK_CGRP, "group.protocol.type", _RK_C_KSTR, | |||
_RK(group_protocol_type), | |||
"Group protocol type for the `generic` group protocol. NOTE: Currently, " | |||
"Group protocol type for the `classic` group protocol. NOTE: Currently, " | |||
"the only supported group " | |||
"protocol type is `consumer`.", | |||
.sdef = "consumer"}, | |||
{_RK_GLOBAL | _RK_CGRP | _RK_HIGH | _RK_HIDDEN, "group.protocol", _RK_C_S2I, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
{_RK_GLOBAL | _RK_CGRP | _RK_HIGH | _RK_HIDDEN, "group.protocol", _RK_C_S2I, | |
{_RK_GLOBAL | _RK_CGRP | _RK_HIGH, "group.protocol", _RK_C_S2I, |
We should unhide these properties now.
src/rdkafka_conf.c
Outdated
.vdef = RD_KAFKA_GROUP_PROTOCOL_GENERIC, | ||
.s2i = {{RD_KAFKA_GROUP_PROTOCOL_GENERIC, "generic"}, | ||
.vdef = RD_KAFKA_GROUP_PROTOCOL_CLASSIC, | ||
.s2i = {{RD_KAFKA_GROUP_PROTOCOL_CLASSIC, "classic"}, | ||
{RD_KAFKA_GROUP_PROTOCOL_CONSUMER, "consumer"}}}, | ||
{_RK_GLOBAL | _RK_CGRP | _RK_MED | _RK_HIDDEN, "group.remote.assignor", |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
{_RK_GLOBAL | _RK_CGRP | _RK_MED | _RK_HIDDEN, "group.remote.assignor", | |
{_RK_GLOBAL | _RK_CGRP | _RK_MED, "group.remote.assignor", |
const rd_kafka_Uuid_t topic_id) { | ||
int i = rd_kafka_topic_partition_list_find_by_id0( | ||
rktparlist, topic_id, RD_KAFKA_PARTITION_UA, | ||
rd_kafka_topic_partition_cmp_topic_id); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There is another function rd_kafka_topic_partition_by_id_cmp
. Lets rename that function to be rd_kafka_topic_partition_by_id_and_partition_cmp
@@ -3117,6 +3117,14 @@ int rd_kafka_topic_partition_cmp_topic(const void *_a, const void *_b) { | |||
return strcmp(a->topic, b->topic); | |||
} | |||
|
|||
/** @brief Compare only the topic id */ | |||
int rd_kafka_topic_partition_cmp_topic_id(const void *_a, const void *_b) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Rename this to rd_kafka_topic_partition_by_id_cmp
to be consistent with other functions.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We have a function named rd_kafka_topic_partition_list_find_by_id
but it finds by id and partition. Let's rename that rd_kafka_topic_partition_list_find_topic_by_id_and_partition
src/rdkafka_cgrp.h
Outdated
/** Send a complete request in next heartbeat, | ||
* but don't send the acknowledgement if it's not required */ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
/** Send a complete request in next heartbeat, | |
* but don't send the acknowledgement if it's not required */ | |
/** Send a complete request in next heartbeat */ |
I don't think statement after but is required.
src/rdkafka_cgrp.h
Outdated
/** Member is fenced, rejoining */ | ||
#define RD_KAFKA_CGRP_CONSUMER_F_WAIT_REJOIN_TO_COMPLETE 0x40 | ||
/** Member is sending an acknowledgement for a reconciled assignment */ | ||
#define RD_KAFKA_CGRP_CONSUMER_F_SENDING_ACK 0x80 |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's move this near RD_KAFKA_CGRP_CONSUMER_F_WAIT_ACK
src/rdkafka_conf.c
Outdated
.vdef = RD_KAFKA_GROUP_PROTOCOL_GENERIC, | ||
.s2i = {{RD_KAFKA_GROUP_PROTOCOL_GENERIC, "generic"}, | ||
.vdef = RD_KAFKA_GROUP_PROTOCOL_CLASSIC, | ||
.s2i = {{RD_KAFKA_GROUP_PROTOCOL_CLASSIC, "classic"}, | ||
{RD_KAFKA_GROUP_PROTOCOL_CONSUMER, "consumer"}}}, | ||
{_RK_GLOBAL | _RK_CGRP | _RK_MED | _RK_HIDDEN, "group.remote.assignor", | ||
_RK_C_STR, _RK(group_remote_assignor), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we should explain what we are doing in the default case (null) as well. Something related to partition assignment strategy. Also, we should explain that this default will be changed in the future so its better to use one of the value from now itself.
src/rdkafka.c
Outdated
if (!rk->rk_conf.group_remote_assignor) { | ||
/* Default remote assignor to the chosen local one. */ | ||
if (rk->rk_conf.partition_assignors_cooperative) { | ||
group_remote_assignor_override = rd_strdup("uniform"); | ||
rk->rk_conf.group_remote_assignor = | ||
rd_strdup("uniform"); | ||
group_remote_assignor_override; | ||
} else { | ||
rd_kafka_assignor_t *range_assignor = | ||
rd_kafka_assignor_find(rk, "range"); | ||
if (range_assignor && range_assignor->rkas_enabled) | ||
rk->rk_conf.group_remote_assignor = | ||
if (range_assignor && range_assignor->rkas_enabled) { | ||
group_remote_assignor_override = | ||
rd_strdup("range"); | ||
rk->rk_conf.group_remote_assignor = | ||
group_remote_assignor_override; | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In the new protocol, there is no significance of the partition assignment strategy. We should not depend on it. If we want to depend on it due to backward compatibility for now, we should tell user that this behaviour will be changed in the future and default remote assignor would be uniform
as suggested in the KIP.
On other though, we shouldn't put any default based on partition assignment strategy and force users to chose one for this protocol. If they want to use the new protocol, they will anyways going to change to group.protocol
property, so why not choose this as well.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
To be able to do a transparent upgrade until next major release we want to detect the methods it's going to use in rebalance_cb, and also give the same assignment results. I'm assuming broker assignors will give the same results in terms of stickyness or rack-awareness.
For roundrobin I think it's best to revert the configuration to the classic protocol, considering the new protocol will be enabled by default it doesn't need to give a fatal error with existing code.
Instead when group.remote.assignor
is set we can assume they start directly with the incremental methods.
if (group_remote_assignor_override) | ||
rd_free(group_remote_assignor_override); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This can be outside of if (app_conf)
condition as we are already doing a NULL check here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if not app_conf
the property if freed in rd_kafka_destroy_internal
so it would be a double free, here it's setting memory to zero later
memset(&rk->rk_conf, 0, sizeof(rk->rk_conf));
so they're not freed by rd_kafka_destroy_internal
and the properties that belonged to app_conf
are still valid pointers that are freed by the app.
d2444a7
to
d25149b
Compare
514851a
to
08b9df5
Compare
08b9df5
to
4eb6f47
Compare
2779b09
to
4b005f0
Compare
65cb3a8
to
dfc2fee
Compare
cd88a71
to
addd3d7
Compare
e517019
to
9ba042b
Compare
addd3d7
to
1411693
Compare
9ba042b
to
bee04c3
Compare
1411693
to
b605960
Compare
bee04c3
to
d0f2a7e
Compare
b605960
to
7e06fa5
Compare
d0f2a7e
to
270f7f7
Compare
7e06fa5
to
92eb02b
Compare
270f7f7
to
228b93d
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
More comments.
@@ -109,7 +109,9 @@ group.instance.id | C | | | |||
partition.assignment.strategy | C | | range,roundrobin | medium | The name of one or more partition assignment strategies. The elected group leader will use a strategy supported by all members of the group to assign partitions to group members. If there is more than one eligible strategy, preference is determined by the order of this list (strategies earlier in the list have higher priority). Cooperative and non-cooperative (eager) strategies must not be mixed. Available strategies: range, roundrobin, cooperative-sticky. <br>*Type: string* | |||
session.timeout.ms | C | 1 .. 3600000 | 45000 | high | Client group session and failure detection timeout. The consumer sends periodic heartbeats (heartbeat.interval.ms) to indicate its liveness to the broker. If no hearts are received by the broker for a group member within the session timeout, the broker will remove the consumer from the group and trigger a rebalance. The allowed range is configured with the **broker** configuration properties `group.min.session.timeout.ms` and `group.max.session.timeout.ms`. Also see `max.poll.interval.ms`. <br>*Type: integer* | |||
heartbeat.interval.ms | C | 1 .. 3600000 | 3000 | low | Group session keepalive heartbeat interval. <br>*Type: integer* | |||
group.protocol.type | C | | consumer | low | Group protocol type for the `generic` group protocol. NOTE: Currently, the only supported group protocol type is `consumer`. <br>*Type: string* | |||
group.protocol.type | C | | consumer | low | Group protocol type for the `classic` group protocol. NOTE: Currently, the only supported group protocol type is `consumer`. <br>*Type: string* | |||
group.protocol | C | classic, consumer | classic | high | Group protocol to use. Use `classic` for the original protocol and `consumer` for the new protocol introduced in KIP-848. Available protocols: classic or consumer. Default is `classic`, but will change to `consumer` in next releases. <br>*Type: enum value* |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
group.protocol | C | classic, consumer | classic | high | Group protocol to use. Use `classic` for the original protocol and `consumer` for the new protocol introduced in KIP-848. Available protocols: classic or consumer. Default is `classic`, but will change to `consumer` in next releases. <br>*Type: enum value* | |
group.protocol | C | classic, consumer | classic | high | Group protocol to use. Available protocols: classic or consumer. Use `classic` for the original protocol and `consumer` for the new protocol introduced in KIP-848. Default is `classic`, but will change to `consumer` in some later release. <br>*Type: enum value* |
src/rdkafka.c
Outdated
if (!rk->rk_conf.group_remote_assignor) { | ||
/* Default remote assignor to the chosen local one. */ | ||
if (rk->rk_conf.partition_assignors_cooperative) { | ||
rk->rk_conf.group_remote_assignor = | ||
rd_strdup("uniform"); | ||
} else { | ||
rd_kafka_assignor_t *range_assignor = | ||
rd_kafka_assignor_find(rk, "range"); | ||
if (range_assignor && range_assignor->rkas_enabled) | ||
rd_kafka_assignor_t *cooperative_assignor; | ||
|
||
/* Detect if chosen assignor is cooperative */ | ||
cooperative_assignor = | ||
rd_kafka_assignor_find(rk, "cooperative-sticky"); | ||
rk->rk_conf.partition_assignors_cooperative = | ||
!rk->rk_conf.partition_assignors.rl_cnt || | ||
(cooperative_assignor && | ||
cooperative_assignor->rkas_enabled); | ||
|
||
if (rk->rk_conf.group_protocol == | ||
RD_KAFKA_GROUP_PROTOCOL_CONSUMER) { | ||
/* Default remote assignor to the chosen local one. */ | ||
if (rk->rk_conf.partition_assignors_cooperative) { | ||
group_remote_assignor_override = | ||
rd_strdup("uniform"); | ||
rk->rk_conf.group_remote_assignor = | ||
rd_strdup("range"); | ||
group_remote_assignor_override; | ||
} else { | ||
rd_kafka_assignor_t *range_assignor = | ||
rd_kafka_assignor_find(rk, "range"); | ||
if (range_assignor && | ||
range_assignor->rkas_enabled) { | ||
group_remote_assignor_override = | ||
rd_strdup("range"); | ||
rk->rk_conf.group_remote_assignor = | ||
group_remote_assignor_override; | ||
} else { | ||
rd_kafka_log( | ||
rk, LOG_WARNING, "ASSIGNOR", | ||
"roundrobin assignor isn't " | ||
"available" | ||
"with group protocol CONSUMER, " | ||
"reverting group protocol " | ||
"to CLASSIC"); | ||
rk->rk_conf.group_protocol = | ||
RD_KAFKA_GROUP_PROTOCOL_CLASSIC; | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's add a FIXME here for removing these changes.
group.protocol.type | C | | consumer | low | Group protocol type for the `generic` group protocol. NOTE: Currently, the only supported group protocol type is `consumer`. <br>*Type: string* | ||
group.protocol.type | C | | consumer | low | Group protocol type for the `classic` group protocol. NOTE: Currently, the only supported group protocol type is `consumer`. <br>*Type: string* | ||
group.protocol | C | classic, consumer | classic | high | Group protocol to use. Use `classic` for the original protocol and `consumer` for the new protocol introduced in KIP-848. Available protocols: classic or consumer. Default is `classic`, but will change to `consumer` in next releases. <br>*Type: enum value* | ||
group.remote.assignor | C | | | medium | Server side assignor to use. Keep it null to make server select a suitable assignor for the group. Available assignors: uniform or range. Default is null <br>*Type: string* |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is incorrect now as we are selecting based on partition.assignment.strategy
. We can update this doc to reflect that information and tell that this is not expected path and will be removed later.
@@ -109,7 +109,9 @@ group.instance.id | C | | | |||
partition.assignment.strategy | C | | range,roundrobin | medium | The name of one or more partition assignment strategies. The elected group leader will use a strategy supported by all members of the group to assign partitions to group members. If there is more than one eligible strategy, preference is determined by the order of this list (strategies earlier in the list have higher priority). Cooperative and non-cooperative (eager) strategies must not be mixed. Available strategies: range, roundrobin, cooperative-sticky. <br>*Type: string* | |||
session.timeout.ms | C | 1 .. 3600000 | 45000 | high | Client group session and failure detection timeout. The consumer sends periodic heartbeats (heartbeat.interval.ms) to indicate its liveness to the broker. If no hearts are received by the broker for a group member within the session timeout, the broker will remove the consumer from the group and trigger a rebalance. The allowed range is configured with the **broker** configuration properties `group.min.session.timeout.ms` and `group.max.session.timeout.ms`. Also see `max.poll.interval.ms`. <br>*Type: integer* | |||
heartbeat.interval.ms | C | 1 .. 3600000 | 3000 | low | Group session keepalive heartbeat interval. <br>*Type: integer* | |||
group.protocol.type | C | | consumer | low | Group protocol type for the `generic` group protocol. NOTE: Currently, the only supported group protocol type is `consumer`. <br>*Type: string* | |||
group.protocol.type | C | | consumer | low | Group protocol type for the `classic` group protocol. NOTE: Currently, the only supported group protocol type is `consumer`. <br>*Type: string* | |||
group.protocol | C | classic, consumer | classic | high | Group protocol to use. Use `classic` for the original protocol and `consumer` for the new protocol introduced in KIP-848. Available protocols: classic or consumer. Default is `classic`, but will change to `consumer` in next releases. <br>*Type: enum value* |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can add that this is the EA release and the APIs and compatibility for consumer
protocol is expected to change.
src/rdkafka.c
Outdated
rd_kafka_log( | ||
rk, LOG_WARNING, "ASSIGNOR", | ||
"roundrobin assignor isn't " | ||
"available" | ||
"with group protocol CONSUMER, " | ||
"reverting group protocol " | ||
"to CLASSIC"); | ||
rk->rk_conf.group_protocol = | ||
RD_KAFKA_GROUP_PROTOCOL_CLASSIC; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reverting to classic is incorrect here. We need to set it to range
or uniform
.
if (rkcg->rkcg_join_state == RD_KAFKA_CGRP_JOIN_STATE_STEADY && | ||
rkcg->rkcg_consumer_flags & RD_KAFKA_CGRP_CONSUMER_F_WAITS_ACK && | ||
(rkcg->rkcg_consumer_flags & RD_KAFKA_CGRP_CONSUMER_F_WAIT_ACK) && | ||
rkcg->rkcg_target_assignment) { | ||
if (rkcg->rkcg_current_assignment) | ||
if (rkcg->rkcg_consumer_flags & | ||
RD_KAFKA_CGRP_CONSUMER_F_SENDING_ACK) { | ||
if (rkcg->rkcg_current_assignment) | ||
rd_kafka_topic_partition_list_destroy( | ||
rkcg->rkcg_current_assignment); | ||
rkcg->rkcg_current_assignment = | ||
rd_kafka_topic_partition_list_copy( | ||
rkcg->rkcg_target_assignment); | ||
rd_kafka_topic_partition_list_destroy( | ||
rkcg->rkcg_current_assignment); | ||
rkcg->rkcg_current_assignment = | ||
rd_kafka_topic_partition_list_copy( | ||
rkcg->rkcg_target_assignment); | ||
rd_kafka_topic_partition_list_destroy( | ||
rkcg->rkcg_target_assignment); | ||
rkcg->rkcg_target_assignment = NULL; | ||
rkcg->rkcg_consumer_flags &= | ||
~RD_KAFKA_CGRP_CONSUMER_F_WAITS_ACK; | ||
if (rd_kafka_is_dbg(rkcg->rkcg_rk, CGRP)) { | ||
char rkcg_current_assignment_str[512] = "NULL"; | ||
rkcg->rkcg_target_assignment); | ||
rkcg->rkcg_target_assignment = NULL; | ||
rkcg->rkcg_consumer_flags &= | ||
~RD_KAFKA_CGRP_CONSUMER_F_WAIT_ACK; | ||
|
||
rd_kafka_topic_partition_list_str( | ||
rkcg->rkcg_current_assignment, | ||
rkcg_current_assignment_str, | ||
sizeof(rkcg_current_assignment_str), 0); | ||
if (rd_kafka_is_dbg(rkcg->rkcg_rk, CGRP)) { | ||
char rkcg_current_assignment_str[512] = "NULL"; | ||
|
||
rd_kafka_dbg( | ||
rkcg->rkcg_rk, CGRP, "HEARTBEAT", | ||
"Target assignment acked, new current assignment " | ||
" \"%s\"", | ||
rkcg_current_assignment_str); | ||
rd_kafka_topic_partition_list_str( | ||
rkcg->rkcg_current_assignment, | ||
rkcg_current_assignment_str, | ||
sizeof(rkcg_current_assignment_str), 0); | ||
|
||
rd_kafka_dbg(rkcg->rkcg_rk, CGRP, "HEARTBEAT", | ||
"Target assignment acked, new " | ||
"current assignment " | ||
" \"%s\"", | ||
rkcg_current_assignment_str); | ||
} | ||
} else if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_SUBSCRIPTION) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Need more understanding on this change.
rkcg->rkcg_last_heartbeat_err = RD_KAFKA_RESP_ERR_NO_ERROR; | ||
~RD_KAFKA_CGRP_CONSUMER_F_SENDING_NEW_SUBSCRIPTION & | ||
~RD_KAFKA_CGRP_CONSUMER_F_SEND_FULL_REQUEST & | ||
~RD_KAFKA_CGRP_CONSUMER_F_SENDING_ACK; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What if the state is not STEADY?
@@ -3033,10 +3089,12 @@ void rd_kafka_cgrp_handle_ConsumerGroupHeartbeat(rd_kafka_t *rk, | |||
/* Re-query for coordinator */ | |||
rkcg->rkcg_consumer_flags |= | |||
RD_KAFKA_CGRP_CONSUMER_F_SEND_FULL_REQUEST; | |||
rd_kafka_cgrp_consumer_expedite_next_heartbeat(rkcg); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shouldn't it be after next line?
} | ||
|
||
/* Skip heartbeat if we have one in transit */ | ||
if (rkcg->rkcg_flags & RD_KAFKA_CGRP_F_HEARTBEAT_IN_TRANSIT) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why removing this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Moved to rd_kafka_cgrp_consumer_heartbeat_preconditions_met
if (rd_kafka_topic_match(rkcg->rkcg_rk, pattern, | ||
topic)) { | ||
break; | ||
if (likely(rkcg->rkcg_heartbeat_intvl_ms > 0)) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Isn't the next timer already set?
- rename 'generic' protocol to 'classic' - consumer group serve timer to awake the loop earlier - compare and find into topic partition list by topic id only - fix memory leak when instance creation fails and app_conf is provided - fix cases where HB response is received after unsubscription - use topic name from current assignment if it's missing from metadata
fixes to next interval check and to the place where number of retries is increased
- expedite HB after changing state back to INIT - use the CONSUMER_F_WAIT_REJOIN to trigger the rejoin instead of calling it from max poll interval timer
reschedule it if expected earlier expedite through scheduling cgrp serve timer
as fatal
immediately, for a possible desynchronization with the coordinator, with said partitions not being consumed anymore
228b93d
to
215ec6e
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM!.. Added couple of comments to track.
rd_kafka_MetadataRequest( | ||
rkb, NULL, topic_ids, "ConsumerGroupHeartbeat API Response", | ||
rkb, NULL, missing_topic_ids, "ConsumerGroupHeartbeat API Response", | ||
rd_false /*!allow_auto_create*/, rd_false, rd_false, rko); | ||
rd_list_destroy(topic_ids); | ||
rd_list_destroy(missing_topic_ids); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just requesting "missing_topics_ids" will not increase cache expiry time for the other topics. Can they be purged inbetween?
rd_kafka_cgrp_coord_query(rkcg, rd_kafka_err2str(err)); | ||
} | ||
|
||
if (actions & RD_KAFKA_ERR_ACTION_RETRY && | ||
rkcg->rkcg_flags & RD_KAFKA_CGRP_F_SUBSCRIPTION && |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Using this flag at many place is error prone. We should improve this flow maybe with Subscription state machine.
- rename 'generic' protocol to 'classic' - consumer group serve timer to awake the loop earlier - compare and find into topic partition list by topic id only - fix memory leak when instance creation fails and app_conf is provided - fix cases where HB response is received after unsubscription - use topic name from current assignment if it's missing from metadata - expedite heartbeat simplification and fixes to next interval check and to the place where number of retries is increased - expedite HB after changing state back to INIT - use the CONSUMER_F_WAIT_REJOIN to trigger the rejoin instead of calling it from max poll interval timer - schedule timer for next execution reschedule it if expected earlier expedite through scheduling cgrp serve timer - treat unsupported feature error as fatal - avoid removing partitions not matched by a new subscription immediately, for a possible desynchronization with the coordinator, with said partitions not being consumed anymore
- rename 'generic' protocol to 'classic' - consumer group serve timer to awake the loop earlier - compare and find into topic partition list by topic id only - fix memory leak when instance creation fails and app_conf is provided - fix cases where HB response is received after unsubscription - use topic name from current assignment if it's missing from metadata - expedite heartbeat simplification and fixes to next interval check and to the place where number of retries is increased - expedite HB after changing state back to INIT - use the CONSUMER_F_WAIT_REJOIN to trigger the rejoin instead of calling it from max poll interval timer - schedule timer for next execution reschedule it if expected earlier expedite through scheduling cgrp serve timer - treat unsupported feature error as fatal - avoid removing partitions not matched by a new subscription immediately, for a possible desynchronization with the coordinator, with said partitions not being consumed anymore