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

Rajini Sivaram commented on KAFKA-18194:
----------------------------------------

Looked at the logs of a failed run. The test has a cluster with only two 
brokers. And the brokers are bounced one after another. When the first broker 
is bounced, consumer gets metadata that contains only the second broker. So 
when the second broker is bounced and the client cannot connect to it, 
rebootstrap is triggered. KIP-899 rebootstrap is performed when there are no 
available brokers in the current metadata. We made this the default in 4.0 
under KIP-1102. The main reason we hit this in the test is because the test 
uses only two brokers and hence results in no available brokers (unlikely 
condition in a production cluster). It will be best to disable rebootstrapping 
in this test using `metadata.recovery.strategy=none` for the consumer, since 
the test specifically expects no metadata change.

> Flaky test_broker_rolling_bounce due to metadata update
> -------------------------------------------------------
>
>                 Key: KAFKA-18194
>                 URL: https://issues.apache.org/jira/browse/KAFKA-18194
>             Project: Kafka
>          Issue Type: Bug
>          Components: consumer, system tests
>            Reporter: Philip Nee
>            Priority: Critical
>
> This is introduced after KIP-1102 I believe this is a critical bug because 
> bounced broker will caused rebalance.
> When examining the consumer log you will see:
>  # When we bounce a broker, sometimes it triggers rebootstrapping
>  # Rebootstrapping forces metadata update
>  # Request joining group due to: cached metadata has changed from (version20: 
> \{test_topic=[NO_RACKS]}) at the beginning of the rebalance to (ver     
> sion21: \{test_topic=[]}) 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> # Causing the consumer to rejoin every time the server is bounded
> Here's the snippet of the log
>  
> {code:java}
> 8156 [2024-12-10 04:44:37,227] DEBUG [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] Sending 
> API_VERSIONS request with header RequestHeader(apiKey=API_VERSIONS, 
> apiVersion=4, clientId=consumer-test_group_id-1, correlationId=1     22, 
> headerVersion=2) and timeout 30000 to node 1: 
> ApiVersionsRequestData(clientSoftwareName='apache-kafka-java', 
> clientSoftwareVersion='---') (org.apache.kafka.clients.NetworkClient)
> 8157 [2024-12-10 04:44:37,227] TRACE [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] Polling for fetches 
> with timeout 1600 
> (org.apache.kafka.clients.consumer.internals.ClassicKafkaConsumer)
> 8158 [2024-12-10 04:44:37,227] TRACE [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] Found least loaded 
> connecting node ducker03:9092 (id: -1 rack: null) 
> (org.apache.kafka.clients.NetworkClient)
> 8159 [2024-12-10 04:44:37,227] TRACE For telemetry state SUBSCRIPTION_NEEDED, 
> returning the value 156047 ms;  
> (org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter)
> 8160 [2024-12-10 04:44:37,227] INFO [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] Request joining 
> group due to: cached metadata has changed from (version20: 
> {test_topic=[NO_RACKS]}
> ) at the beginning of the rebalance to (ver     sion21:
> {test_topic=[]}
> ) (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> 8161 [2024-12-10 04:44:37,227] DEBUG [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] Executing 
> onJoinPrepare with generation 7 and memberId 
> consumer-test_group_id-1-61dfefb9-acdd-4050-ba0a-56f5d5ed1c3d 
> (org.apache.kafka.clien     ts.consumer.internals.ConsumerCoordinator)
> 8162 [2024-12-10 04:44:37,227] INFO [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] Revoke previously 
> assigned partitions  
> (org.apache.kafka.clients.consumer.internals.ConsumerRebalanceListenerInvoker)
> 8163 [2024-12-10 04:44:37,228] INFO [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] (Re-)joining group 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> 8164 [2024-12-10 04:44:37,228] DEBUG [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] Joining group with 
> current subscription: [test_topic] 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> 8165 [2024-12-10 04:44:37,228] DEBUG [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] Sending JoinGroup 
> (JoinGroupRequestData(groupId='test_group_id', sessionTimeoutMs=30000, 
> rebalanceTimeoutMs=300000, memberId='consumer-test_     
> group_id-1-61dfefb9-acdd-4050-ba0a-56f5d5ed1c3d', groupInstanceId=null, 
> protocolType='consumer', protocols=[JoinGroupRequestProtocol(name='range', 
> metadata=[0, 3, 0, 0, 0, 1, 0, 10, 116, 101, 115, 116, 95, 116, 111, 112, 
> 105, 99, -1, -1, -1     , -1, 0, 0, 0, 0, 0, 0, 0, 7, -1, -1])], 
> reason='cached metadata has changed')) to coordinator ducker03:9092 (id: 
> 2147483646 rack: null) 
> (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> 8166 [2024-12-10 04:44:37,228] DEBUG [Consumer 
> clientId=consumer-test_group_id-1, groupId=test_group_id] Sending JOIN_GROUP 
> request with header RequestHeader(apiKey=JOIN_GROUP, apiVersion=9, 
> clientId=consumer-test_group_id-1, correlationId=123,      headerVersion=2) 
> and timeout 305000 to node 2147483646: 
> JoinGroupRequestData(groupId='test_group_id', sessionTimeoutMs=30000, 
> rebalanceTimeoutMs=300000, 
> memberId='consumer-test_group_id-1-61dfefb9-acdd-4050-ba0a-56f5d5ed1c3d', 
> groupInstance     Id=null, protocolType='consumer', 
> protocols=[JoinGroupRequestProtocol(name='range', metadata=[0, 3, 0, 0, 0, 1, 
> 0, 10, 116, 101, 115, 116, 95, 116, 111, 112, 105, 99, -1, -1, -1, -1, 0, 0, 
> 0, 0, 0, 0, 0, 7, -1, -1])], reason='cached metadat     a has changed') 
> (org.apache.kafka.clients.NetworkClient)
> {code}
>  #  Please reproduce this running 
> {code:java}
> TC_PATHS="tests/kafkatest/tests/client/consumer_test.py::OffsetValidationTest.test_broker_failure"
>  bash tests/docker/run_tests.sh {code}



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

Reply via email to