Skip to content

Commit d38a90d

Browse files
authored
KAFKA-17672 Run quarantined tests separately (#17329)
Introduce new quarantinedTest that excludes tests tagged with "flaky". Also introduce two new build parameters "maxQuarantineTestRetries" and "maxQuarantineTestRetryFailures". Reviewers: Chia-Ping Tsai <[email protected]>
1 parent f1d6549 commit d38a90d

File tree

7 files changed

+130
-14
lines changed

7 files changed

+130
-14
lines changed

.github/workflows/build.yml

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -130,18 +130,19 @@ jobs:
130130
timeout ${TIMEOUT_MINUTES}m ./gradlew --build-cache --continue --no-scan \
131131
-PtestLoggingEvents=started,passed,skipped,failed \
132132
-PmaxParallelForks=2 \
133-
-PmaxTestRetries=1 -PmaxTestRetryFailures=10 \
133+
-PmaxTestRetries=1 -PmaxTestRetryFailures=3 \
134+
-PmaxQuarantineTestRetries=3 -PmaxQuarantineTestRetryFailures=0 \
134135
-PcommitId=xxxxxxxxxxxxxxxx \
135-
test
136+
quarantinedTest test
136137
exitcode="$?"
137138
echo "exitcode=$exitcode" >> $GITHUB_OUTPUT
138-
- name: Archive JUnit reports
139+
- name: Archive JUnit HTML reports
139140
uses: actions/upload-artifact@v4
140141
id: junit-upload-artifact
141142
with:
142143
name: junit-reports-${{ matrix.java }}
143144
path: |
144-
**/build/reports/tests/test/*
145+
**/build/reports/tests/*
145146
compression-level: 9
146147
if-no-files-found: ignore
147148
- name: Archive Thread Dumps

.github/workflows/deflake.yml

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -60,7 +60,9 @@ jobs:
6060
-PtestLoggingEvents=started,passed,skipped,failed \
6161
-PignoreFailures=true -PmaxParallelForks=2 \
6262
-Pkafka.cluster.test.repeat=${{ inputs.test-repeat }} \
63-
${{ inputs.test-module }}:test --tests ${{ inputs.test-pattern }}
63+
-PmaxTestRetries=${{ inputs.test-repeat }} -PmaxTestRetryFailures=0 \
64+
-PmaxQuarantineTestRetries=${{ inputs.test-repeat }} -PmaxQuarantineTestRetryFailures=0 \
65+
${{ inputs.test-module }}:test ${{ inputs.test-module }}:quarantinedTest --tests ${{ inputs.test-pattern }}
6466
exitcode="$?"
6567
echo "exitcode=$exitcode" >> $GITHUB_OUTPUT
6668
- name: Archive JUnit reports

README.md

Lines changed: 16 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -34,14 +34,16 @@ Follow instructions in https://kafka.apache.org/quickstart
3434
./gradlew docsJar # builds both (if applicable) javadoc and scaladoc jars for each module
3535

3636
### Run unit/integration tests ###
37-
./gradlew test # runs both unit and integration tests
37+
./gradlew test # runs both unit and integration tests
3838
./gradlew unitTest
3939
./gradlew integrationTest
40+
./gradlew quarantinedTest # runs the quarantined tests
41+
4042

4143
### Force re-running tests without code change ###
42-
./gradlew test --rerun
43-
./gradlew unitTest --rerun
44-
./gradlew integrationTest --rerun
44+
./gradlew test --rerun-tasks
45+
./gradlew unitTest --rerun-tasks
46+
./gradlew integrationTest --rerun-tasks
4547

4648
### Running a particular unit/integration test ###
4749
./gradlew clients:test --tests RequestResponseTest
@@ -64,11 +66,17 @@ to `log4j.logger.org.apache.kafka=INFO` and then run:
6466
And you should see `INFO` level logs in the file under the `clients/build/test-results/test` directory.
6567

6668
### Specifying test retries ###
67-
By default, each failed test is retried once up to a maximum of five retries per test run. Tests are retried at the end of the test task. Adjust these parameters in the following way:
69+
By default, each failed test is retried once up to a maximum of three total retries per test run.
70+
Tests are retried at the end of the test task. Adjust these parameters in the following way:
6871

69-
./gradlew test -PmaxTestRetries=1 -PmaxTestRetryFailures=5
70-
71-
See [Test Retry Gradle Plugin](https://github.com/gradle/test-retry-gradle-plugin) for more details.
72+
./gradlew test -PmaxTestRetries=1 -PmaxTestRetryFailures=3
73+
74+
Additionally, quarantined tests are automatically retried three times up to a total of
75+
20 retries per run. This is controlled by similar parameters.
76+
77+
./gradlew test -PmaxQuarantineTestRetries=3 -PmaxQuarantineTestRetryFailures=20
78+
79+
See [Test Retry Gradle Plugin](https://github.com/gradle/test-retry-gradle-plugin) for and [build.yml](.github/workflows/build.yml) more details.
7280

7381
### Generating test coverage reports ###
7482
Generate coverage reports for the whole project:

build.gradle

Lines changed: 62 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -85,6 +85,9 @@ ext {
8585
userMaxTestRetries = project.hasProperty('maxTestRetries') ? maxTestRetries.toInteger() : 0
8686
userMaxTestRetryFailures = project.hasProperty('maxTestRetryFailures') ? maxTestRetryFailures.toInteger() : 0
8787

88+
userMaxQuarantineTestRetries = project.hasProperty('maxQuarantineTestRetries') ? maxQuarantineTestRetries.toInteger() : 0
89+
userMaxQuarantineTestRetryFailures = project.hasProperty('maxQuarantineTestRetryFailures') ? maxQuarantineTestRetryFailures.toInteger() : 0
90+
8891
skipSigning = project.hasProperty('skipSigning') && skipSigning.toBoolean()
8992
shouldSign = !skipSigning && !version.endsWith("SNAPSHOT")
9093

@@ -501,6 +504,7 @@ subprojects {
501504

502505
useJUnitPlatform {
503506
includeEngines 'junit-jupiter'
507+
excludeTags 'flaky'
504508
}
505509

506510
develocity {
@@ -524,7 +528,7 @@ subprojects {
524528
if (ext.isGithubActions) {
525529
def dest = rootProject.layout.buildDirectory.dir("junit-xml/${project.name}").get().asFile
526530
println "Copy JUnit XML for ${project.name} to $dest"
527-
ant.copy(todir: "$dest") {
531+
ant.copy(todir: "$dest/test") {
528532
ant.fileset(dir: "${test.reports.junitXml.entryPoint}")
529533
}
530534

@@ -537,6 +541,62 @@ subprojects {
537541
}
538542
}
539543

544+
task quarantinedTest(type: Test, dependsOn: compileJava) {
545+
ext {
546+
isGithubActions = System.getenv('GITHUB_ACTIONS') != null
547+
}
548+
549+
// Disable caching and up-to-date for this task. We always want quarantined tests
550+
// to run and never want to cache their results. Since we do this, we can avoid
551+
// explicitly failing the build like we do in "test" with ext.hadFailure.
552+
outputs.upToDateWhen { false }
553+
outputs.cacheIf { false }
554+
555+
maxParallelForks = maxTestForks
556+
ignoreFailures = userIgnoreFailures
557+
558+
maxHeapSize = defaultMaxHeapSize
559+
jvmArgs = defaultJvmArgs
560+
561+
// KAFKA-17433 Used by deflake.yml github action to repeat individual tests
562+
systemProperty("kafka.cluster.test.repeat", project.findProperty("kafka.cluster.test.repeat"))
563+
564+
testLogging {
565+
events = userTestLoggingEvents ?: testLoggingEvents
566+
showStandardStreams = userShowStandardStreams ?: testShowStandardStreams
567+
exceptionFormat = testExceptionFormat
568+
displayGranularity = 0
569+
}
570+
logTestStdout.rehydrate(delegate, owner, this)()
571+
572+
useJUnitPlatform {
573+
includeEngines 'junit-jupiter'
574+
includeTags 'flaky'
575+
}
576+
577+
develocity {
578+
testRetry {
579+
maxRetries = userMaxQuarantineTestRetries
580+
maxFailures = userMaxQuarantineTestRetryFailures
581+
}
582+
}
583+
584+
// This closure will copy JUnit XML files out of the sub-project's build directory and into
585+
// a top-level build/junit-xml directory. This is necessary to avoid reporting on tests which
586+
// were not run, but instead were restored via FROM-CACHE. See KAFKA-17479 for more details.
587+
doLast {
588+
if (ext.isGithubActions) {
589+
def dest = rootProject.layout.buildDirectory.dir("junit-xml/${project.name}").get().asFile
590+
println "Copy JUnit XML for ${project.name} to $dest"
591+
ant.copy(todir: "$dest/quarantinedTest", failonerror: "false") {
592+
ant.fileset(dir: "${quarantinedTest.reports.junitXml.entryPoint}") {
593+
ant.include(name: "**/*.xml")
594+
}
595+
}
596+
}
597+
}
598+
}
599+
540600
task integrationTest(type: Test, dependsOn: compileJava) {
541601
maxParallelForks = maxTestForks
542602
ignoreFailures = userIgnoreFailures
@@ -3507,6 +3567,7 @@ project(':connect:runtime') {
35073567
testImplementation project(':storage')
35083568
testImplementation project(':connect:test-plugins')
35093569
testImplementation project(':server-common').sourceSets.test.output
3570+
testImplementation project(':test-common:test-common-api')
35103571

35113572
testImplementation libs.junitJupiter
35123573
testImplementation libs.mockitoCore

connect/runtime/src/test/java/org/apache/kafka/connect/integration/OffsetsApiIntegrationTest.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919
import org.apache.kafka.clients.CommonClientConfigs;
2020
import org.apache.kafka.clients.admin.Admin;
2121
import org.apache.kafka.clients.admin.ConsumerGroupListing;
22+
import org.apache.kafka.common.test.api.Flaky;
2223
import org.apache.kafka.connect.runtime.ConnectorConfig;
2324
import org.apache.kafka.connect.runtime.SourceConnectorConfig;
2425
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
@@ -331,6 +332,7 @@ public void testAlterSinkConnectorOffsets() throws Exception {
331332
alterAndVerifySinkConnectorOffsets(baseSinkConnectorConfigs(), connect.kafka());
332333
}
333334

335+
@Flaky("KAFKA-15914")
334336
@Test
335337
public void testAlterSinkConnectorOffsetsOverriddenConsumerGroupId() throws Exception {
336338
Map<String, String> connectorConfigs = baseSinkConnectorConfigs();

core/src/test/scala/unit/kafka/coordinator/transaction/ProducerIdManagerTest.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import org.apache.kafka.common.errors.CoordinatorLoadInProgressException
2424
import org.apache.kafka.common.message.AllocateProducerIdsResponseData
2525
import org.apache.kafka.common.protocol.Errors
2626
import org.apache.kafka.common.requests.AllocateProducerIdsResponse
27+
import org.apache.kafka.common.test.api.Flaky
2728
import org.apache.kafka.common.utils.{MockTime, Time}
2829
import org.apache.kafka.server.NodeToControllerChannelManager
2930
import org.apache.kafka.server.common.ProducerIdsBlock
@@ -40,6 +41,7 @@ import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch, Executors, T
4041
import scala.collection.mutable
4142
import scala.util.{Failure, Success, Try}
4243

44+
@Flaky("KAFKA-17654")
4345
class ProducerIdManagerTest {
4446

4547
var brokerToController: NodeToControllerChannelManager = mock(classOf[NodeToControllerChannelManager])
Lines changed: 40 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,40 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.kafka.common.test.api;
19+
20+
import org.junit.jupiter.api.Tag;
21+
22+
import java.lang.annotation.ElementType;
23+
import java.lang.annotation.Retention;
24+
import java.lang.annotation.RetentionPolicy;
25+
import java.lang.annotation.Target;
26+
27+
@Target({ElementType.TYPE, ElementType.METHOD})
28+
@Retention(RetentionPolicy.RUNTIME)
29+
@Tag("flaky")
30+
public @interface Flaky {
31+
/**
32+
* Required reference to a KAFKA Jira ticket.
33+
*/
34+
String value();
35+
36+
/**
37+
* Optional comment describing the reason for quarantined.
38+
*/
39+
String comment() default "";
40+
}

0 commit comments

Comments
 (0)