[ 
https://issues.apache.org/jira/browse/KAFKA-18212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17904842#comment-17904842
 ] 

Chia-Ping Tsai commented on KAFKA-18212:
----------------------------------------

{quote}
during rolling restart with multiple brokers, other brokers would be available
{quote}

It seems that unexpected metadata changes could occur in the following case. 
Let's assume we are performing a rolling upgrade on a two-broker cluster:

1. Restart broker_0
2. broker_1 marks broker_0 as fenced
3. Consumer fetches metadata, which includes only broker_1 (active)
4. broker_0 comes back online
5. Restart broker_1
6. Consumer detects broker_1 disconnection and finds no available nodes (least 
load node) – begins rebootstrap - metadata change

In fact, the above case is almost equal to KAFKA-8206. 

{quote}
That would be removing the rebootstrapping path introduced in KIP-899? 
{quote}

The true issue is - Should we just rebootstarp metadata or wait some time 
(metadata.recovery.rebootstrap.trigger.ms) when there is no available nodes (we 
don't know disconnection is temporary or forever) ? The side effect of using 
rebootstarp is in rolling-upgrade it could produce unnecessary re-balance (due 
to metadata change). By contrast, honoring the 
metadata.recovery.rebootstrap.trigger.ms results in more time (5 mins) to wait.

Maybe another way is we rely just on `metadata.recovery.rebootstrap.trigger.ms` 
but the default value should be reduced from 300s to 60s

> Rolling-upgrade broker causes unnecessary re-balance on classic consumer
> ------------------------------------------------------------------------
>
>                 Key: KAFKA-18212
>                 URL: https://issues.apache.org/jira/browse/KAFKA-18212
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Chia-Ping Tsai
>            Assignee: Chia-Ping Tsai
>            Priority: Blocker
>
> KAFKA-17885 sets the rebootstrap policy by default and introduces 
> metadata.recovery.rebootstrap.trigger.ms to control the rebootstrap interval.
> However, DefaultMetadataUpdater#maybeUpdate [0] does not honor 
> metadata.recovery.rebootstrap.trigger.ms and triggers a rebootstrap when no 
> node is available. This causes the metadata to be reset during rolling 
> upgrades of brokers, leading the classic consumer [1] to send a rejoin 
> request due to the metadata change.
> [0] 
> https://github.com/apache/kafka/blob/7591868aead54fff7d5e8a44c5e06746ed34866b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java#L1230
> [1] 
> https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L883



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

Reply via email to