Skip to content

KAFKA-14108: Ensure both JUnit 4 and JUnit 5 tests run#12441

Merged
cadonna merged 5 commits intoapache:trunkfrom
clolov:KAFKA-7342-ENABLE-TESTS
Jul 29, 2022
Merged

KAFKA-14108: Ensure both JUnit 4 and JUnit 5 tests run#12441
cadonna merged 5 commits intoapache:trunkfrom
clolov:KAFKA-7342-ENABLE-TESTS

Conversation

@clolov
Copy link
Contributor

@clolov clolov commented Jul 26, 2022

This pull request addresses the problem reported in #12285 and tracked in https://issues.apache.org/jira/browse/KAFKA-14108

@divijvaidya
Copy link
Member

@clolov

  1. don't we need to remove "streams" from
  def shouldUseJUnit5 = !(["runtime", "streams"].contains(it.project.name))

as well?

  1. please add the test report for "before" the change and for "after" the change. This would help us validate that number of tests run "after" this change is greater than or equal to "before" the change.

@cadonna
Copy link
Member

cadonna commented Jul 26, 2022

Thanks for the PR, @clolov !

The builds have errors:

* What went wrong:
[2022-07-26T12:41:05.041Z] A problem occurred evaluating root project 'Kafka_kafka-pr_PR-12441'.
[2022-07-26T12:41:05.041Z] > Cannot convert a null value to an object of type Dependency.
[2022-07-26T12:41:05.041Z]   The following types/formats are supported:
[2022-07-26T12:41:05.041Z]     - Instances of Dependency.
[2022-07-26T12:41:05.041Z]     - String or CharSequence values, for example 'org.gradle:gradle-core:1.0'.
[2022-07-26T12:41:05.041Z]     - Maps, for example [group: 'org.gradle', name: 'gradle-core', version: '1.0'].
[2022-07-26T12:41:05.041Z]     - FileCollections, for example files('some.jar', 'someOther.jar').
[2022-07-26T12:41:05.041Z]     - Projects, for example project(':some:project:path').
[2022-07-26T12:41:05.041Z]     - ClassPathNotation, for example gradleApi().

build.gradle Outdated
// KAFKA-14109
// Both engines are needed to run JUnit 4 tests alongside JUnit 5 tests.
// junit-vintage (JUnit 4) can be removed once the JUnit 4 migration is complete.
includeEngines "junit-vintage", "junit-jupiter"
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't want to do this for all modules since most of them have already been converted.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a disadvantage/side-effect if we do it for all the modules?

From what I understand (please correct me if I am wrong), loading the vintage engine for JUnit5 tests won't have any side effects since they would be run with jupiter engine anyways. Vintage engine only runs Junit4 tests in a Junit5 platform. It does not impact already converted Junit5 tests running on Junit5 platform.

I am advocating for this because it keeps the changes minimal and simplified here. Given that it is a temporary transient stage (we already have PRs out for most of the test conversion to JUnit5), I would preferr minimal changes so that reverting them is easier.

@cadonna
Copy link
Member

cadonna commented Jul 26, 2022

@clolov I also found a mistake in #12285:
In KTableSourceTopicRestartIntegrationTest line 103 should be

sourceTopic = SOURCE_TOPIC + "-" + testInfo.getTestMethod().map(Method::getName).orElse("");`

instead of

sourceTopic = SOURCE_TOPIC + "-" + testInfo.getTestMethod().map(Method::getName);

otherwise the test does not run wince the topic name contains illegal characters.

build.gradle Outdated
if (shouldUseJUnit5) {
useJUnitPlatform {
includeTags "integration"
// KAFKA-14109
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We don't typically include jira references in the code like this (it pollutes the code and goes stale typically).

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I second Ismael's comment.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed the comment in the latest revision.

// KAFKA-14109
// The below dependency is needed for compiling JUnit 4 tests.
// It can be safely removed once all of streams has moved to JUnit 5.
junit4: "junit:junit:$versions.junit4",
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This explanation is odd - compilation works currently and we don't have this dependency.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did some tests and discovered that junitVintageEngine pulls in JUnit 4. That means if we move junitVintageEngine back to the testImplementation dependencies compilation of JUnit4 tests works.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That makes sense. There should be no need for depending on the old junit4 jars directly.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed the dependency in latest revision.

Copy link
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@clolov Thanks for the PR!

I had some suggestions how you can ensure that all tests are run without changing already migrated modules.

build.gradle Outdated
if (shouldUseJUnit5) {
useJUnitPlatform {
includeTags "integration"
// KAFKA-14109
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I second Ismael's comment.

build.gradle Outdated
Comment on lines 467 to 473
useJUnitPlatform {
includeTags "integration"
// KAFKA-14109
// Both engines are needed to run JUnit 4 tests alongside JUnit 5 tests.
// junit-vintage (JUnit 4) can be removed once the JUnit 4 migration is complete.
includeEngines "junit-vintage", "junit-jupiter"
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I experimented a bit and found the following solution.

Suggested change
useJUnitPlatform {
includeTags "integration"
// KAFKA-14109
// Both engines are needed to run JUnit 4 tests alongside JUnit 5 tests.
// junit-vintage (JUnit 4) can be removed once the JUnit 4 migration is complete.
includeEngines "junit-vintage", "junit-jupiter"
}
if (project.name.equals('streams')) {
useJUnitPlatform {
includeTags "integration"
includeTags 'org.apache.kafka.test.IntegrationTest'
includeEngines "junit-vintage", "junit-jupiter"
}
} else {
useJUnitPlatform {
includeTags "integration"
}
}

In this way, we can limit the change to Streams.
The caveat is that we need to replace @Category({IntegrationTest.class}) with @Tag{"integration"} in the integration tests that have already been migrated to JUnit 5.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Made the changes as suggested. Please see latest revision.

build.gradle Outdated
Comment on lines 508 to 514
useJUnitPlatform {
excludeTags "integration"
// KAFKA-14109
// Both engines are needed to run JUnit 4 tests alongside JUnit 5 tests.
// junit-vintage (JUnit 4) can be removed once the JUnit 4 migration is complete.
includeEngines "junit-vintage", "junit-jupiter"
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here we can do similar as above but excluding the tags.

Suggested change
useJUnitPlatform {
excludeTags "integration"
// KAFKA-14109
// Both engines are needed to run JUnit 4 tests alongside JUnit 5 tests.
// junit-vintage (JUnit 4) can be removed once the JUnit 4 migration is complete.
includeEngines "junit-vintage", "junit-jupiter"
}
if (project.name.equals('streams')) {
useJUnitPlatform {
excludeTags "integration"
excludeTags 'org.apache.kafka.test.IntegrationTest'
includeEngines "junit-vintage", "junit-jupiter"
}
} else {
useJUnitPlatform {
excludeTags "integration"
}
}

build.gradle Outdated
Comment on lines 1843 to 1847
// KAFKA-14109
// The below compileOnly dependency is needed for JUnit 4 tests.
// It can be safely removed once all of streams has moved to JUnit 5.
testCompileOnly libs.junit4

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As we said below this is not needed if you move libs.junitVintageEngine back to testImplementation.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed this dependency.

build.gradle Outdated
testImplementation libs.junitJupiterApi
testImplementation libs.junitVintageEngine
testImplementation libs.junitJupiter
testImplementation libs.junitJupiterParams // needed for parameterized tests
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would prefer to add this when we need it during the migration of the parametrized tests. But I am also fine if it stays.

The inline comment is not needed.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have removed this right now to keep this PR simple. Will add when needed.

@divijvaidya
Copy link
Member

Thanks for the comments @cadonna @ijuma. Christo is on vacation right now (we are team mates) but I will take up this PR and file a revision addressing your comments tomorrow.

@cadonna
Copy link
Member

cadonna commented Jul 28, 2022

@divijvaidya That would be great, because I start feeling uncomfortable having some integration tests not running on the builds.
Please also make sure to compare test reports before and after the change as you proposed.
Thanks a lot for taking over this work for Christo when he is on vacation!

@cadonna
Copy link
Member

cadonna commented Jul 28, 2022

@divijvaidya Regarding your question

don't we need to remove "streams" from def shouldUseJUnit5 = !(["runtime", "streams"].contains(it.project.name)) as well?

Yes, we need.

@divijvaidya
Copy link
Member

divijvaidya commented Jul 29, 2022

Changes

  1. Replace with @Category({IntegrationTest.class}) with @Tag("integration")
  2. Unblock streams project from using JUnitPlatform.
  3. Use junit-vintage and junit-jupiter engine for executing tests of streams project. Tests formatted as Junit4 will automatically use junit-vintage and tests formatted as JUnit5 will automatically use junit-jupiter.
  4. Minor change required in KTableSourceTopicRestartIntegrationTest.java after using junit-jupiter

Results

Result of executing ./gradlew :streams:test before & after this change.

Before

Screenshot 2022-07-29 at 11 30 16

After

Screenshot 2022-07-29 at 11 20 44

Note

  1. After this change we are running more number of tests. This is because tests which were migrated to JUnit 5 were not being executed earlier.
  2. **./gradlew :streams:integrationTest runs lesser number of tests than earlier. It only runs tests which have been converted to JUnit5. The other integration tests are run as UnitTests instead. I am trying to figure out why this is happening. **

Comments

Addressed concerns from @ijuma:

  • regarding not affecting other projects with this change (note that junit-vintage is being used for streams only)
  • regarding removal of addition of unnecessary dependencies (removed the unnecessary dependency)

Addressed concerns from @cadonna

  • regarding adding junitJupiterParams dependency only when needed (removed the dependency)

Next steps (separate PRs, in order)

  1. Migrate tests using PowerMock to use Mockito
  2. Migrate all tests to Junit5
  3. Remove the Junit4 dependency completely from build.gradle

@cadonna
Copy link
Member

cadonna commented Jul 29, 2022

@divijvaidya Thank you for the updates!

I am afraid you did a mistake when generating the test reports to test the changes. You changed task integrationTest and task unitTest but you generated the reports with streams:test. In task test no filtering is done on the tags integration and org.apache.kafka.test.IntegrationTest. Also the builds use integrationTest and unitTest to run the tests and not test. You should generate the reports with streams:integrationTest and streams:unitTest.

In my tests yesterday, I experienced that you cannot use @Tags("integration") (and filter on it) on integration tests that are written JUnit 4, but you need to leave @Category({IntegrationTest.class}) in the tests and use includeTags "org.apache.kafka.test.IntegrationTest" in the build file. For integration tests written in JUnit 5, you need to use @Tags("integration") in the tests and includeTags "integration" in the build file. See my comment #12441 (comment).

@BeforeEach
public void before(final TestInfo testInfo) throws Exception {
sourceTopic = SOURCE_TOPIC + "-" + testInfo.getTestMethod().map(Method::getName);
sourceTopic = SOURCE_TOPIC + "-" + IntegrationTestUtils.safeUniqueTestName(getClass(), testInfo);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note that the test fails without this change.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, thank you! That is known. See #12441 (comment)

@divijvaidya
Copy link
Member

divijvaidya commented Jul 29, 2022

Updated the code. @cadonna should be ready for your review once the test run is complete.

Result for ./gradlew :streams:integrationTest

Before

Screenshot 2022-07-29 at 13 54 02

After

Screenshot 2022-07-29 at 13 54 52

Result for ./gradlew :streams:unitTest

Before

Screenshot 2022-07-29 at 13 57 36

After

Screenshot 2022-07-29 at 13 56 39

@cadonna
Copy link
Member

cadonna commented Jul 29, 2022

@divijvaidya Thanks a lot for the update! This looks good!

However, there are some checkstyle issues due to some unused imports. Could you fix those?

Copy link
Member

@cadonna cadonna left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @divijvaidya and @clolov !

LGTM!

Once the builds are acceptable and I verified that they run the tests written in JUnit 5, I will merge this PR.

@cadonna
Copy link
Member

cadonna commented Jul 29, 2022

I verified that all Streams' tests are run in the builds and I also verified that the tests of other modules are run in the builds.

@cadonna
Copy link
Member

cadonna commented Jul 29, 2022

Build failures are unrelated.

@cadonna cadonna merged commit 54af64c into apache:trunk Jul 29, 2022
ijuma added a commit to confluentinc/kafka that referenced this pull request Aug 5, 2022
…(5 August 2022)

Version related conflicts:
* Jenkinsfile
* gradle.properties
* streams/quickstart/java/pom.xml
* streams/quickstart/java/src/main/resources/archetype-resources/pom.xml
* streams/quickstart/pom.xml
* tests/kafkatest/__init__.py
* tests/kafkatest/version.py

* commit 'add7cd85baa61cd0e1430': (66 commits)
KAFKA-14136 Generate ConfigRecord for brokers even if the value is
unchanged (apache#12483)
  HOTFIX / KAFKA-14130: Reduce RackAwarenesssTest to unit Test (apache#12476)
  MINOR: Remove ARM/PowerPC builds from Jenkinsfile (apache#12380)
  KAFKA-14111 Fix sensitive dynamic broker configs in KRaft (apache#12455)
  KAFKA-13877: Fix flakiness in RackAwarenessIntegrationTest (apache#12468)
KAFKA-14129: KRaft must check manual assignments for createTopics are
contiguous (apache#12467)
KAFKA-13546: Do not fail connector validation if default topic
creation group is explicitly specified (apache#11615)
KAFKA-14122: Fix flaky test
DynamicBrokerReconfigurationTest#testKeyStoreAlter (apache#12452)
  MINOR; Use right enum value for broker registration change (apache#12236)
  MINOR; Synchronize access to snapshots' TreeMap (apache#12464)
  MINOR; Bump trunk to 3.4.0-SNAPSHOT (apache#12463)
  MINOR: Stop logging 404s at ERROR level in Connect
KAFKA-14095: Improve handling of sync offset failures in MirrorMaker
(apache#12432)
  Minor: enable index for emit final sliding window (apache#12461)
  MINOR: convert some more junit tests to support KRaft (apache#12456)
  KAFKA-14108: Ensure both JUnit 4 and JUnit 5 tests run (apache#12441)
  MINOR: Remove code of removed metric (apache#12453)
MINOR: Update comment on verifyTaskGenerationAndOwnership method in
DistributedHerder
KAFKA-14012: Add warning to closeQuietly documentation about method
references of null objects (apache#12321)
  MINOR: Fix static mock usage in ThreadMetricsTest (apache#12454)
  ...
@clolov clolov deleted the KAFKA-7342-ENABLE-TESTS branch January 27, 2026 11:37
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants