KAFKA-17925: Convert Kafka Client integration tests to use KRaft#17670
KAFKA-17925: Convert Kafka Client integration tests to use KRaft#17670chia7712 merged 16 commits into
Conversation
Update pertinent integration tests to use KRaft and not Zookeeper.
|
it seems there are some failed tests caused by this PR |
Update to only test CLASSIC group protocol
| val gp = maybeGroupProtocolSpecified() | ||
|
|
||
| if (gp.isEmpty) | ||
| throw new IllegalStateException("Please specify the group.protocol configuration when creating a KafkaConsumer") |
There was a problem hiding this comment.
maybeGroupProtocolSpecified is based on the input arguments of test case. Maybe the error message should be "Please specify the groupProtocol=consumer when writing the test"
There was a problem hiding this comment.
I updated the error message to Please specify the "groupProtocol" parameter when writing the test. I didn't use groupProtocol=consumer in the error message because the value of groupProtocol can be classic or consumer.
PTAL. Thanks!
| def testTopicIdIsRemovedFromFetcherWhenControllerDowngrades(): Unit = { | ||
| @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) | ||
| @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit")) | ||
| def testTopicIdIsRemovedFromFetcherWhenControllerDowngrades(quorum: String, groupProtocol: String): Unit = { |
There was a problem hiding this comment.
This test case is for the ZK-only API (LeaderAndIsrRequest). Not sure if we plan to remove all ZK-only APIs in 4.0... cc @dajac.
There was a problem hiding this comment.
There are around 20 integration tests that are effectively Zookeeper-only. In some cases, nothing in the test explicitly calls that dependency out. I introduced the @MethodSource getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit() which includes the test parameters quorum=zk and groupProtocol=classic. The idea is that someone more knowledgeable about the tests would review these and either a) remove the test, or b) update the test to run against Kraft, if possible.
There was a problem hiding this comment.
The idea is that someone more knowledgeable about the tests would review these and either a) remove the test, or b) update the test to run against Kraft, if possible.
Yes, that will be a great reference, similar to #17727.
For example, FetchRequestTestDowngrade is for LeaderAndIsrRequest, which is ZK-only, so it's okay to remove it in a follow-up (https://issues.apache.org/jira/browse/KAFKA-17976).
|
@kirktrue Could you please fix the conflicts and comment (#17670 (comment))? I'd like to merge this PR |
|
|
|
Hey folks, I noticed |
|
@lianetm we were just looking at this from our end. I think it depends on what is failing. Looking more closely into this now. |
|
As an aside, if we introduce a new test and/or version of a test and we see it fail in the build, we should not merge the PR. |
|
Found the issue and will open a fix shortly -- it was on the transactions side and not the group coordinator side. :) EDIT: PR here: #17831 |
|
Great, thanks @jolshan! |
You're right. I'll make sure to address those flaky tests next time. |
…he#17670) Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
* KAFKA-17926 Improve the documentation explaining why max.in.flight.requests.per.connection should not exceed 5 (apache#17719) Reviewers: Chia-Ping Tsai <chia7712@gmail.com> * MINOR: Cleanup GroupCoordinatorRecordHelpers (apache#17718) Reviewers: Jeff Kim <jeff.kim@confluent.io>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com> * rebase to fix merge conflict (apache#17702) Fixes an issue with the TTD in the specific case where users don't specify an initial time for the driver and also don't specify a start timestamp for the TestInputTopic, then pipe input records without timestamps. This combination results in a slight mismatch in the expected timestamps for the piped records, which can be noticeable when writing tests with very small time deltas. The problem is that, while both the TTD and the TestInputTopic will be initialized to the "current time" when not otherwise specified, it's possible for some milliseconds to have passed between the creation of the TTD and the creation of the TestInputTopic. This can result in a TestInputTopic getting a start timestamp that's several ms larger than the driver's time, and ultimately causing the piped input records to have timestamps slightly in the future relative to the driver. In practice even those who hit this issue might not notice it if they aren't manipulating time in their tests, or are advancing time by enough to negate the several-milliseconds of difference. However we noticed a test fail due to this because we were testing a ttl-based processor and had advanced the driver time by only 1 millisecond past the ttl. The piped record should have been expired, but because it's timestamp was a few milliseconds longer than the driver's start time, this test ended up failing. Reviewers: Matthias Sax <mjsax@apache.org>, Bruno Cadonna <cadonna@apache.org>, Lucas Brutschy < lbrutschy@confluent.io> * KAFKA-17801: RemoteLogManager may compute inaccurate upperBoundOffset for aborted txns (apache#17676) Reviewers: Jun Rao <junrao@gmail.com> * MINOR: log fix in SnapshottableCoordinator (apache#17726) Reviewers: donaldzhu-cc, Chia-Ping Tsai <chia7712@gmail.com> * KAFKA-17570 Rewrite StressTestLog by Java (apache#17249) Reviewers: Chia-Ping Tsai <chia7712@gmail.com> * MINOR: Delete unused member from KafkaAdminClient (apache#17729) Reviewers: Chia-Ping Tsai <chia7712@gmail.com> * KAFKA-17970 Moving some share purgatory classes from core to share module (apache#17722) As part of PR: apache#17636 where purgatory has been moved from core to server-common hence move some existing classes used in Share Fetch to Share module. Reviewers: Chia-Ping Tsai <chia7712@gmail.com> * KAFKA-17837 Rewrite DeleteTopicTest (apache#17579) Reviewers: Chia-Ping Tsai <chia7712@gmail.com> * MINOR: Move StopPartition to server-common (apache#17704) Reviewers: Chia-Ping Tsai <chia7712@gmail.com> * KAFKA-15549 Bump swagger dependency version from 2.2.8 to 2.2.25 (apache#17730) Reviewers: Chia-Ping Tsai <chia7712@gmail.com> * KAFKA-17872: Update consumed offsets on records with invalid timestamp (apache#17710) TimestampExtractor allows to drop records by returning a timestamp of -1. For this case, we still need to update consumed offsets to allows us to commit progress. Reviewers: Bill Bejeck <bill@confluent.io> * KAFKA-17925 Convert Kafka Client integration tests to use KRaft (apache#17670) Reviewers: Chia-Ping Tsai <chia7712@gmail.com> * KAFKA-17779: Fix flaky RemoteLogManager test (apache#17724) Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com> * KAFKA-17455: fixes stuck producer by polling again after pollDelayMs in NetworkUtils#awaitReady() * clarifies comments * attempts to add test * Adds a test but my changes to MockClient.java broke all sorts of stuff * test that passes on my branch and fails on trunk * addresses PR feedback: rename MockClient#setAdvanceTimeDuringPoll to advanceTimeDuringPoll() --------- Co-authored-by: PoAn Yang <payang@apache.org> Co-authored-by: David Jacot <djacot@confluent.io> Co-authored-by: A. Sophie Blee-Goldman <ableegoldman@gmail.com> Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com> Co-authored-by: Jeff Kim <kimkb2011@gmail.com> Co-authored-by: TengYao Chi <kitingiao@gmail.com> Co-authored-by: Apoorv Mittal <apoorvmittal10@gmail.com> Co-authored-by: Mickael Maison <mimaison@users.noreply.github.com> Co-authored-by: Yung <yungyung7654321@gmail.com> Co-authored-by: Matthias J. Sax <matthias@confluent.io> Co-authored-by: Kirk True <kirk@kirktrue.pro> Co-authored-by: wperlichek <61857706+wperlichek@users.noreply.github.com> Co-authored-by: Colt McNealy <colt@littlehorse.io>
…he#17670) Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Update pertinent integration tests to use KRaft and not Zookeeper.
Committer Checklist (excluded from commit message)