KAFKA-14498: reduce the startup nodes to avoid timeout error (#13016)

In MetadataQuorumCommandTest, we sometimes got the error:

java.util.concurrent.ExecutionException: java.lang.RuntimeException: Received a fatal error while waiting for the broker to catch up with the current cluster metadata.

Since we tried to bring up 3 broker + 3 controllers at the same time, and the config initial.broker.registration.timeout.ms (default 1 min) is sometimes not enough for them to start up. Checking the tests, it doesn't require so many nodes. Reducing the nodes number to make these tests reliable.

Reviewers: dengziming <dengziming1993@gmail.com>, Ismael Juma <ismael@juma.me.uk>
This commit is contained in:
Luke Chen 2022-12-21 11:19:22 +08:00 committed by GitHub
parent c4f10364cd
commit 2575362639
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 13 additions and 13 deletions

View File

@ -53,12 +53,12 @@ class MetadataQuorumCommandTest {
* 3. Fewer brokers than controllers
*/
@ClusterTests({
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3),
@ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3),
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2),
@ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2),
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3),
@ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3)
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 2),
@ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 2),
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 1),
@ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 1),
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 1, controllers = 2),
@ClusterTest(clusterType = Type.KRAFT, brokers = 1, controllers = 2)
})
public void testDescribeQuorumReplicationSuccessful() throws InterruptedException {
cluster.waitForReadyBrokers();
@ -94,12 +94,12 @@ class MetadataQuorumCommandTest {
* 3. Fewer brokers than controllers
*/
@ClusterTests({
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 3),
@ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 3),
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 3, controllers = 2),
@ClusterTest(clusterType = Type.KRAFT, brokers = 3, controllers = 2),
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 3),
@ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 3)
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 2),
@ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 2),
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 2, controllers = 1),
@ClusterTest(clusterType = Type.KRAFT, brokers = 2, controllers = 1),
@ClusterTest(clusterType = Type.CO_KRAFT, brokers = 1, controllers = 2),
@ClusterTest(clusterType = Type.KRAFT, brokers = 1, controllers = 2)
})
public void testDescribeQuorumStatusSuccessful() throws InterruptedException {
cluster.waitForReadyBrokers();
@ -141,7 +141,7 @@ class MetadataQuorumCommandTest {
assertEquals("0", replicationOutput.split("\n")[1].split("\\s+")[2]);
}
@ClusterTest(clusterType = Type.ZK, brokers = 3)
@ClusterTest(clusterType = Type.ZK, brokers = 1)
public void testDescribeQuorumInZkMode() {
assertTrue(
assertThrows(