You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/03/15 21:51:01 UTC

[GitHub] [kafka] rondagostino opened a new pull request #10322: KAFKA-12455: OffsetValidationTest.test_broker_rolling_bounce fail: Raft

rondagostino opened a new pull request #10322:
URL: https://github.com/apache/kafka/pull/10322


   OffsetValidationTest.test_broker_rolling_bounce was failing when used with a Raft-based metadata quorum but succeeding with a ZooKeeper-based quorum.  The problem was that in the Raft case the consumer was sometimes receiving a METADATA response with just 1 alive broker, and then when that broker rolled the consumer wouldn't know about any alive nodes.  It would have to wait until the broker returned before it could reconnect, and by that time the group coordinator on the second broker would have timed-out the client and initiated a group rebalance.  The test explicitly checks that no rebalances occur, so the test would fail.  It turns out that the reason why the ZooKeeper configuration wasn't seeing rebalances was just plain luck.  The brokers' metadata caches in the ZooKeeper configuration show 1 alive broker even more frequently than the Raft configuration does.  If we tweak the metadata.max.age.ms value on the consumers we can easily get the ZooKeeper test to fail, and in fact
  this system test has historically been flaky for the ZoKeeper configuration.  We can get the test to pass by setting session.timeout.ms=30000 (which is longer than the roll time of any broker), or we can increase the broker count so that the client never sees a METADATA response with just a single alive broker and therefore never loses contact with the cluster for an extended period of time.
   
   This patch increases the broker count for this particular system test from 2 to 3.
   
   This patch also fixes a minor logging bug that was discovered in `RaftReplicaManager` that was discovered during the debugging of this issue, and it adds an extra logging statement when a single metadata batch is applied to mirror the same logging statement that occurs when deferred metadata changes are applied.
   
   
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on pull request #10322: KAFKA-12455: OffsetValidationTest.test_broker_rolling_bounce fail: Raft

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #10322:
URL: https://github.com/apache/kafka/pull/10322#issuecomment-799777289


   This patch needs to be cherry-picked to 2.8


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on pull request #10322: KAFKA-12455: OffsetValidationTest.test_broker_rolling_bounce fail: Raft

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #10322:
URL: https://github.com/apache/kafka/pull/10322#issuecomment-800356645


   @ijuma Thanks for the suggestions -- all set I think.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] rondagostino commented on pull request #10322: KAFKA-12455: OffsetValidationTest.test_broker_rolling_bounce fail: Raft

Posted by GitBox <gi...@apache.org>.
rondagostino commented on pull request #10322:
URL: https://github.com/apache/kafka/pull/10322#issuecomment-799781073


   As per an offline conversation, since 2 brokers is a supported cluster size, we would prefer that this system test keep 2 brokers instead of bumping it to 3 -- we have lots of tests that run with 3 brokers already.  So I will change the test to use the `session.timeout.ms=30000` solution instead.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma merged pull request #10322: KAFKA-12455: OffsetValidationTest.test_broker_rolling_bounce fail: Raft

Posted by GitBox <gi...@apache.org>.
ijuma merged pull request #10322:
URL: https://github.com/apache/kafka/pull/10322


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #10322: KAFKA-12455: OffsetValidationTest.test_broker_rolling_bounce fail: Raft

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #10322:
URL: https://github.com/apache/kafka/pull/10322#discussion_r595144123



##########
File path: tests/kafkatest/tests/client/consumer_test.py
##########
@@ -93,6 +93,7 @@ def test_broker_rolling_bounce(self, metadata_quorum=quorum.zk):
         partition = TopicPartition(self.TOPIC, 0)
 
         producer = self.setup_producer(self.TOPIC)
+        self.session_timeout_sec = 30

Review comment:
       Can we please add a comment here explaining why this is important?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org