mirror of https://github.com/apache/kafka.git
KAFKA-18025 Remove flaky tag from several tests in ShareConsumerTest (#18226)
Develocity report: https://ge.apache.org/scans/tests?search.relativeStartTime=P28D&search.rootProjectNames=kafka&search.tags=github,trunk&search.timeZoneId=America%2FNew_York&tests.container=kafka.test.api.ShareConsumerTest# Reviewers: David Arthur <mumrah@gmail.com>
This commit is contained in:
parent
337fb8cec1
commit
e88fede27e
|
@ -275,7 +275,6 @@ public class ShareConsumerTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Flaky("KAFKA-18033")
|
||||
@ParameterizedTest(name = "{displayName}.persister={0}")
|
||||
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
|
||||
public void testAcknowledgementSentOnSubscriptionChange(String persister) throws ExecutionException, InterruptedException {
|
||||
|
@ -599,7 +598,6 @@ public class ShareConsumerTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Flaky("KAFKA-18033")
|
||||
@ParameterizedTest(name = "{displayName}.persister={0}")
|
||||
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
|
||||
public void testExplicitAcknowledgementCommitAsync(String persister) throws InterruptedException {
|
||||
|
@ -656,7 +654,6 @@ public class ShareConsumerTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Flaky("KAFKA-18033")
|
||||
@ParameterizedTest(name = "{displayName}.persister={0}")
|
||||
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
|
||||
public void testExplicitAcknowledgementCommitAsyncPartialBatch(String persister) {
|
||||
|
@ -763,7 +760,6 @@ public class ShareConsumerTest {
|
|||
}
|
||||
}
|
||||
|
||||
@Flaky("KAFKA-18033")
|
||||
@ParameterizedTest(name = "{displayName}.persister={0}")
|
||||
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
|
||||
public void testExplicitAcknowledgeReleaseClose(String persister) {
|
||||
|
@ -1188,7 +1184,6 @@ public class ShareConsumerTest {
|
|||
assertEquals(producerCount * messagesPerProducer, totalSuccessResult);
|
||||
}
|
||||
|
||||
@Flaky("KAFKA-18025")
|
||||
@ParameterizedTest(name = "{displayName}.persister={0}")
|
||||
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
|
||||
public void testAcquisitionLockTimeoutOnConsumer(String persister) throws InterruptedException {
|
||||
|
@ -1296,7 +1291,6 @@ public class ShareConsumerTest {
|
|||
* Test to verify that the acknowledgement commit callback can invoke KafkaShareConsumer.wakeup() and it
|
||||
* wakes up the enclosing poll.
|
||||
*/
|
||||
@Flaky("KAFKA-18033")
|
||||
@ParameterizedTest(name = "{displayName}.persister={0}")
|
||||
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
|
||||
public void testAcknowledgementCommitCallbackCallsShareConsumerWakeup(String persister) throws InterruptedException {
|
||||
|
@ -1349,7 +1343,6 @@ public class ShareConsumerTest {
|
|||
* Test to verify that the acknowledgement commit callback can throw an exception, and it is propagated
|
||||
* to the caller of poll().
|
||||
*/
|
||||
@Flaky("KAFKA-18033")
|
||||
@ParameterizedTest(name = "{displayName}.persister={0}")
|
||||
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
|
||||
public void testAcknowledgementCommitCallbackThrowsException(String persister) throws InterruptedException {
|
||||
|
|
Loading…
Reference in New Issue