mirror of https://github.com/apache/kafka.git
KAFKA-18568: Fix flaky test ClientIdQuotaTest (#18612)
The reason for flakiness is PR #18080 which modifies the linger.ms config from 0 to 5. ClientIdQuotaTest are testing "Low enough quota that a producer sending a small payload in a tight loop should get throttled", thus this config change Influence this test scenario. This commits uses the older value of 0ms for linger.ms for ClientIdQuotaTest tests. Reviewers: Ismael Juma <ismael@juma.me.uk>, TaiJuWu <tjwu1217@gmail.com>, Divij Vaidya <diviv@amazon.com>
This commit is contained in:
parent
a842c02b88
commit
71495a2013
|
|
@ -61,6 +61,7 @@ abstract class BaseQuotaTest extends IntegrationTestHarness {
|
|||
this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "-1")
|
||||
this.producerConfig.setProperty(ProducerConfig.BUFFER_MEMORY_CONFIG, "300000")
|
||||
this.producerConfig.setProperty(ProducerConfig.CLIENT_ID_CONFIG, producerClientId)
|
||||
this.producerConfig.setProperty(ProducerConfig.LINGER_MS_CONFIG, 0.toString)
|
||||
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "QuotasTest")
|
||||
this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString)
|
||||
this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
|
||||
|
|
|
|||
Loading…
Reference in New Issue