[ 
https://issues.apache.org/jira/browse/KAFKA-17237?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Emanuele Sabellico updated KAFKA-17237:
---------------------------------------
    Description: 
When configuring a client for rack-awareness to enable FFF and rack-aware 
assignors, a rebalance is triggered every time a broker disappears from a 
Metadata response, such as during a cluster roll.
That happens because after KIP 881 metadata appears as changed given the set of 
racks is different (brokers that are down have no info about the rack).

*How to reproduce*
 * Enable *client.rack* on the client and *broker.rack* on the cluster with a 
different value on each node
 * Create a topic with replicas on all the nodes
 * Subscribe to that topic on the client
 * Stop one of the brokers
 * Observe a rebalance is triggered

Attached is a log reproducing the issue in Java client. A few lines showing the 
rejoin requests
{noformat}
[2024-08-01 15:09:07,472] INFO [Consumer clientId=consumer-test_racks-1, 
groupId=test_racks] Request joining group due to: cached metadata has changed 
from (version4: {test_new=[racks=[null, 1b, 1c]]}) at the beginning of the 
rebalance to (version5: {test_new=[racks=[1a, 1b, 1c]]}) 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2024-08-01 15:10:38,689] INFO [Consumer clientId=consumer-test_racks-1, 
groupId=test_racks] Request joining group due to: cached metadata has changed 
from (version6: {test_new=[racks=[1a, 1b, 1c]]}) at the beginning of the 
rebalance to (version42: {test_new=[racks=[null, 1a, 1c]]}) 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2024-08-01 15:11:04,106] INFO [Consumer clientId=consumer-test_racks-1, 
groupId=test_racks] Request joining group due to: cached metadata has changed 
from (version43: {test_new=[racks=[1a, 1b, 1c]]}) at the beginning of the 
rebalance to (version45: {test_new=[racks=[null, 1a, 1b]]}) 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
{noformat}
Same happens in librdkafka as reported in this issue
[https://github.com/confluentinc/librdkafka/issues/4742]

  was:
When configuring a client for rack-awareness to enable FFF and rack-aware 
assignors, a rebalance is triggered every time a broker disappears from a 
Metadata response, such as during a cluster roll.
That happens because after KIP 881 metadata appears as changed given the set of 
racks is different (brokers that are down have no info about the rack).

*How to reproduce*
 * Enable *client.rack* on the client and *broker.rack* on the cluster
 * Create a topic with replicas on all the nodes
 * Subscribe to that topic on the client
 * Stop one of the brokers
 * Observe a rebalance is triggered

Attached is a log reproducing the issue in Java client. A few lines showing the 
rejoin requests
{noformat}
[2024-08-01 15:09:07,472] INFO [Consumer clientId=consumer-test_racks-1, 
groupId=test_racks] Request joining group due to: cached metadata has changed 
from (version4: {test_new=[racks=[null, 1b, 1c]]}) at the beginning of the 
rebalance to (version5: {test_new=[racks=[1a, 1b, 1c]]}) 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2024-08-01 15:10:38,689] INFO [Consumer clientId=consumer-test_racks-1, 
groupId=test_racks] Request joining group due to: cached metadata has changed 
from (version6: {test_new=[racks=[1a, 1b, 1c]]}) at the beginning of the 
rebalance to (version42: {test_new=[racks=[null, 1a, 1c]]}) 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
[2024-08-01 15:11:04,106] INFO [Consumer clientId=consumer-test_racks-1, 
groupId=test_racks] Request joining group due to: cached metadata has changed 
from (version43: {test_new=[racks=[1a, 1b, 1c]]}) at the beginning of the 
rebalance to (version45: {test_new=[racks=[null, 1a, 1b]]}) 
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
{noformat}
Same happens in librdkafka as reported in this issue
[https://github.com/confluentinc/librdkafka/issues/4742]


> [rack-aware assignors] Rebalance is triggered every time a broker isn't 
> reported from a metadata call
> -----------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-17237
>                 URL: https://issues.apache.org/jira/browse/KAFKA-17237
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients
>    Affects Versions: 3.5.0, 3.8.0
>            Reporter: Emanuele Sabellico
>            Priority: Minor
>         Attachments: test.log
>
>
> When configuring a client for rack-awareness to enable FFF and rack-aware 
> assignors, a rebalance is triggered every time a broker disappears from a 
> Metadata response, such as during a cluster roll.
> That happens because after KIP 881 metadata appears as changed given the set 
> of racks is different (brokers that are down have no info about the rack).
> *How to reproduce*
>  * Enable *client.rack* on the client and *broker.rack* on the cluster with a 
> different value on each node
>  * Create a topic with replicas on all the nodes
>  * Subscribe to that topic on the client
>  * Stop one of the brokers
>  * Observe a rebalance is triggered
> Attached is a log reproducing the issue in Java client. A few lines showing 
> the rejoin requests
> {noformat}
> [2024-08-01 15:09:07,472] INFO [Consumer clientId=consumer-test_racks-1, 
> groupId=test_racks] Request joining group due to: cached metadata has changed 
> from (version4: {test_new=[racks=[null, 1b, 1c]]}) at the beginning of the 
> rebalance to (version5: {test_new=[racks=[1a, 1b, 1c]]}) 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> [2024-08-01 15:10:38,689] INFO [Consumer clientId=consumer-test_racks-1, 
> groupId=test_racks] Request joining group due to: cached metadata has changed 
> from (version6: {test_new=[racks=[1a, 1b, 1c]]}) at the beginning of the 
> rebalance to (version42: {test_new=[racks=[null, 1a, 1c]]}) 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> [2024-08-01 15:11:04,106] INFO [Consumer clientId=consumer-test_racks-1, 
> groupId=test_racks] Request joining group due to: cached metadata has changed 
> from (version43: {test_new=[racks=[1a, 1b, 1c]]}) at the beginning of the 
> rebalance to (version45: {test_new=[racks=[null, 1a, 1b]]}) 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> {noformat}
> Same happens in librdkafka as reported in this issue
> [https://github.com/confluentinc/librdkafka/issues/4742]



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to