Commit Graph

12952 Commits

Author SHA1 Message Date
Kamal Chandraprakash 8655094e6c
KAFKA-16511: Fix the leaking tiered segments during segment deletion (#15817)
When there are overlapping segments in the remote storage, then the deletion may fail to remove the segments due to isRemoteSegmentWithinLeaderEpochs check. Once the deletion starts to fail for a partition, then segments won't be eligible for cleanup. The one workaround that we have is to move the log-start-offset using the kafka-delete-records script.

Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2024-05-08 15:21:23 +08:00
yuz10 5f933ac840
MINOR: Correct connector scheduled rebalance logs (#15875)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-08 12:32:47 +08:00
TingIāu "Ting" Kì a0f1658bb1
KAFKA-16678 Remove variable "unimplementedquorum" (#15879)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-08 12:30:34 +08:00
vamossagar12 525b9b1d76
KAFKA-15018: Write connector tombstone offsets to secondary store before primary store (#13801)
Reviewers: Chris Egerton <chrise@aiven.io>
2024-05-07 14:29:02 -04:00
Jorge Esteban Quilcate Otoya a4c6cefd10
KAFKA-14226: Introduce FieldPath abstraction and nested path support for ExtractField SMT (#15379)
Reviewers: Chris Egerton <chrise@aiven.io>
2024-05-07 14:07:18 -04:00
Chris Egerton 05df10449e
KAFKA-13328, KAFKA-13329 (2): Add custom preflight validation support for connector header, key, and value converters (#14309)
Reviewers:  Greg Harris <greg.harris@aiven.io>
2024-05-07 11:30:57 -04:00
Jeff Kim 21bf715622
KAFKA-16307; Fix coordinator thread idle ratio (#15835)
This PR fixes the thread idle ratio. We take a similar approach to the kafka request handler idle ratio: https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/KafkaRequestHandler.scala#L108-L117

Instead of calculating the actual ratio per thread, we record the time each thread stays idle while waiting for a new event, divided by the number of threads as an approximation.

Reviewers: David Jacot <djacot@confluent.io>
2024-05-07 06:21:09 -07:00
Bruno Cadonna cb35ddc5ca
KAFKA-10199: Remove lost tasks in state updater with new remove (#15870)
Uses the new remove operation of the state updater that returns a future to remove lost tasks from the state udpater.

Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
2024-05-07 14:26:23 +02:00
Lianet Magrans ea485a7061
KAFKA-16665: Allow to initialize newly assigned partition's positions without allowing fetching while callback runs (#15856)
Fix to allow to initialize positions for newly assigned partitions, while the onPartitionsAssigned callback is running, even though the partitions remain non-fetchable until the callback completes.

Before this PR, we were not allowing initialization or fetching while the callback was running. The fix here only allows to initialize the newly assigned partition position, and keeps the existing logic for making sure that the partition remains non-fetchable until the callback completes.

The need for this fix came out in one of the connect system tests, that attempts to retrieve a newly assigned partition position with a call to consumer.position from within the onPartitionsAssigned callback (WorkerSinkTask). With this PR, we allow to make such calls (test added), which is the behaviour of the legacy consumer.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-05-07 10:40:00 +02:00
Dongnuo Lyu 459eaec666
KAFKA-16615; JoinGroup API for upgrading ConsumerGroup (#15798)
The patch implements JoinGroup API for the new consumer groups. It allow members using the classic rebalance protocol with the consumer embedded protocol to join a new consumer group.

Reviewers: David Jacot <djacot@confluent.io>
2024-05-06 23:59:10 -07:00
Matthias J. Sax d76352e215
MINOR: log newly created processId (#15851)
Reviewers: Colt McNealy <colt@littlehorse.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-07 14:14:35 +08:00
TingIāu "Ting" Kì 0de3b7c40b
KAFKA-16593 Rewrite DeleteConsumerGroupsTest by ClusterTestExtensions (#15766)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-07 14:04:32 +08:00
David Jacot 0df340d64d
KAFKA-16470 kafka-dump-log --offsets-decoder should support new records (#15652)
When the consumer group protocol is used in a cluster, it is, at the moment, impossible to see all records stored in the __consumer_offsets topic with kafka-dump-log --offsets-decoder. It does not know how to handle all the new records.

This patch refactors the OffsetsMessageParser used internally by kafka-dump-log to use the RecordSerde used by the new group coordinator. It ensures that the tool is always in sync with the coordinator implementation. The patch also changes the format to using the toString'ed representations of the records instead of having custom logic to dump them. It ensures that all the information is always dumped. The downside of the latest is that inner byte arrays (e.g. assignment in the classic protocol) are no longer deserialized. Personally, I feel like that it is acceptable and it is actually better to stay as close as possible to the actual records in this tool. It also avoids issues like https://issues.apache.org/jira/browse/KAFKA-15603.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-07 08:49:31 +08:00
David Arthur fe8ccbc92c
KAFKA-16539 Fix IncrementalAlterConfigs during ZK migration (#15744)
This patch fixes two issues with IncrementalAlterConfigs and the ZK migration. First, it changes the handling of IncrementalAlterConfigs to check if the controller is ZK vs KRaft and only forward for KRaft. Second, it adds a check in KafkaZkClient#setOrCreateEntityConfigs to ensure a ZK broker is not directly modifying configs in ZK if there is a KRaft controller. This closes the race condition between KRaft taking over as the active controller and the ZK brokers learning about this.

*Forwarding*

During the ZK migration, there is a time when the ZK brokers are running with migrations enabled, but KRaft has yet to take over as the controller. Prior to KRaft taking over as the controller, the ZK brokers in migration mode were unconditionally forwarding IncrementalAlterConfigs (IAC) to the ZK controller. This works for some config types, but breaks when setting BROKER and BROKER_LOGGER configs for a specific broker. The behavior in KafkaApis for IAC was to always forward if the forwarding manager was defined. Since ZK brokers in migration mode have forwarding enabled, the forwarding would happen, and the special logic for BROKER and BROKER_LOGGER would be missed, causing the request to fail.

With this fix, the IAC handler will check if the controller is KRaft or ZK and only forward for KRaft.

*Protected ZK Writes*

As part of KIP-500, we moved most (but not all) ZK mutations to the ZK controller. One of the things we did not move fully to the controller was entity configs. This is because there was some special logic that needed to run on the broker for certain config updates. If a broker-specific config was set, AdminClient would route the request to the proper broker. In KRaft, we have a different mechanism for handling broker-specific config updates.

Leaving this ZK update on the broker side would be okay if we were guarding writes on the controller epoch, but it turns out KafkaZkClient#setOrCreateEntityConfigs does unprotected "last writer wins" updates to ZK. This means a ZK broker could update the contents of ZK after the metadata had been migrated to KRaft. No good! To fix this, this patch adds a check on the controller epoch to KafkaZkClient#setOrCreateEntityConfigs but also adds logic to fail the update if the controller is a KRaft controller.

The new logic in setOrCreateEntityConfigs adds STALE_CONTROLLER_EPOCH as a new exception that can be thrown while updating configs.

Reviewers:  Luke Chen <showuon@gmail.com>, Akhilesh Chaganti <akhileshchg@users.noreply.github.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-07 08:29:57 +08:00
Andrew Schofield 4c4ae6e39c
KAFKA-16608 Honour interrupted thread state on KafkaConsumer.poll (#15803)
The contract of KafkaConsumer.poll(Duration) says that it throws InterruptException "if the calling thread is interrupted before or while this function is called". The new KafkaConsumer implementation was not doing this if the thread was interrupted before the poll was called, specifically with a very short timeout. If it ever waited for records, it did check the thread state. If it did not wait for records because of a short timeout, it did not.

Some of the log messages in the code erroneously mentioned timeouts, when they really meant interruption.

Also adds a test for this specific scenario.

Reviewers: Lianet Magrans <lianetmr@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-07 08:22:41 +08:00
Linu Shibu aeca384641
KAFKA-16356: Remove class-name dispatch in RemoteLogMetadataSerde (#15620)
Reviewers: Greg Harris <greg.harris@aiven.io>, Luke Chen <showuon@gmail.com>, Igor Soarez <soarez@apple.com>, The-Gamer-01 <19974361760@163.com>
2024-05-06 16:49:35 -07:00
Vedarth Sharma 0b4eaefd86
KAFKA-16646: KAFKA-16646Don't run cve scan job on forks (#15831)
Minor fix PR to ensure scan job runs only on apache/kafka

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>
2024-05-06 20:16:07 +05:30
David Jacot 42754336e1
MINOR: Remove `ConsumerGroupPartitionMetadataValue.Epoch` field (#15854)
ConsumerGroupPartitionMetadataValue.Epoch is not used anywhere so we can remove it. Note that we already have non-backward compatible changes lined up for 3.8 so it is fine to do it.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-06 05:02:39 -07:00
Okada Haruki 5c96ad61d9
KAFKA-16393 read/write sequence of buffers correctly (#15571)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-06 19:11:04 +08:00
Nikolay 6a8977e212
KAFKA-14588 [3/N] ConfigCommandTest rewritten in java (#15850)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-06 18:44:34 +08:00
Bruno Cadonna 366aeab488
KAFKA-10199: Add remove operation with future to state updater (#15852)
Adds a remove operation to the state updater that returns a future
instead of adding the removed tasks to an output queue. Code that
uses the state updater can then wait on the future.

Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
2024-05-06 11:27:40 +02:00
Chia Chuan Yu 55a00be4e9
MINOR: Replaced Utils.join() with JDK API. (#15823)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-06 15:13:01 +08:00
Hector Geraldino 41f5bf844d
KAFKA-16223 Replace EasyMock/PowerMock with Mockito for KafkaConfigBackingStoreTest (2/3) (#15841)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-06 08:47:17 +08:00
PoAn Yang 970ac07881
KAFKA-16659 KafkaConsumer#position() does not respect wakup when group protocol is CONSUMER (#15853)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-06 08:45:11 +08:00
Johnny Hsu 25118cec14
MINOR: remove redundant check in KafkaClusterTestKit (#15858)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-05 11:47:40 +08:00
José Armando García Sancio bfe81d6229
KAFKA-16207; KRaft's internal log listener to update voter set (#15671)
Adds support for the KafkaRaftClient to read the control records KRaftVersionRecord and VotersRecord in the snapshot and log. As the control records in the KRaft partition are read, the replica's known set of voters are updated. This change also contains the necessary changes to include the control records when a snapshot is generated by the KRaft state machine.

It is important to note that this commit changes the code and the in-memory state to track the sets of voters but it doesn't change any data that is externally exposed. It doesn't change the RPCs, data stored on disk or configuration.

When the KRaft replica starts the PartitionListener reads the latest snapshot and then log segments up to the LEO, updating the in-memory state as it reads KRaftVersionRecord and VotersRecord. When the replica (leader and follower) appends to the log, the PartitionListener catches up to the new LEO. When the replica truncates the log because of a diverging epoch, the PartitionListener also truncates the in-memory state to the new LEO. When the state machine generate a new snapshot the PartitionListener trims any prefix entries that are not needed. This is all done to minimize the amount of data tracked in-memory and to make sure that it matches the state on disk.

To implement the functionality described above this commit also makes the following changes:

Adds control records for KRaftVersionRecord and VotersRecord. KRaftVersionRecord describes the finalized kraft.version supported by all of the replicas. VotersRecords describes the set of voters at a specific offset.

Changes Kafka's feature version to support 0 as the smallest valid value. This is needed because the default value for kraft.version is 0.

Refactors FileRawSnapshotWriter so that it doesn't directly call the onSnapshotFrozen callback. It adds NotifyingRawSnapshotWriter for calling such callbacks. This reorganization is needed because in this change both the KafkaMetadataLog and the KafkaRaftClient need to react to snapshots getting frozen.

Cleans up KafkaRaftClient's initialization. Removes initialize from RaftClient - this is an implementation detail that doesn't need to be exposed in the interface. Removes RaftConfig.AddressSpec and simplifies the bootstrapping of the static voter's address. The bootstrapping of the address is delayed because of tests. We should be able to simplify this further in future commits.

Update the DumpLogSegment CLI to support the new control records KRaftVersionRecord and VotersRecord.

Fix the RecordsSnapshotReader implementations so that the iterator includes control records. RecordsIterator is extended to support reading the new control records.
Improve the BatchAccumulator implementation to allow multiple control records in one control batch. This is needed so that KRaft can make sure that VotersRecord is included in the same batch as the control record (KRaftVersionRecord) that upgrades the kraft.version to 1.

Add a History interface and default implementation TreeMapHistory. This is used to track all of the sets of voters between the latest snapshot and the LEO. This is needed so that KafkaRaftClient can query for the latest set of voters and so that KafkaRaftClient can include the correct set of voters when the state machine generates a new snapshot at a given offset.

Add a builder pattern for RecordsSnapshotWriter. The new builder pattern also implements including the KRaftVersionRecord and VotersRecord control records in the snapshot as necessary. A KRaftVersionRecord should be appended if the kraft.version is greater than 0 at the snapshot's offset. Similarly, a VotersRecord should be appended to the snapshot with the latest value up to the snapshot's offset.

Reviewers: Jason Gustafson <jason@confluent.io>
2024-05-04 12:43:16 -07:00
Kirk True 9b8aac22ec
KAFKA-16427 KafkaConsumer#position() does not respect timeout when group protocol is CONSUMER (#15843)
The AsyncKafkaConsumer implementation of position(TopicPartition, Duration) was not updating its internal Timer, causing it to execute the loop forever. Adding a call to update the Timer at the bottom of the loop fixes the issue.

An integration test was added to catch this case; it fails without the newly added call to Timer.update(long).

Reviewers: Lianet Magrans <lianetmr@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-04 10:29:27 +08:00
Alyssa Huang 1fd39150aa
KAFKA-16655: Deflake ZKMigrationIntegrationTest.testDualWrite #15845
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Johnny Hsu <44309740+johnnychhsu@users.noreply.github.com>
2024-05-03 10:44:37 -07:00
David Jacot 2c0b8b6920
MINOR: ConsumerGroup#getOrMaybeCreateMember should not add the member to the group (#15847)
While reviewing https://github.com/apache/kafka/pull/15785, I noticed that the member is added to the group directly in `ConsumerGroup#getOrMaybeCreateMember`. This does not hurt but confuses people because the state must not be mutated at this point. It should only be mutated when records are replayed. I think that it is better to remove it in order to make it clear.

Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-03 06:24:26 -07:00
Bruno Cadonna 240243b91d
KAFKA-10199: Accept only one task per element in output queue for failed tasks (#15849)
Currently, the state updater writes multiple tasks per exception in the output
queue for failed tasks. To add the functionality to remove tasks synchronously
from the state updater, it is simpler that each element of the output queue for
failed tasks holds one single task.

This commit refactors the class that holds exceptions and failed tasks
in the state updater -- i.e., ExceptionAndTasks -- to just hold one single
task.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-05-03 10:52:12 +02:00
PoAn Yang 87390f961f
KAFKA-16572 allow defining number of disks per broker in ClusterTest (#15745)
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-03 14:24:59 +08:00
Colin Patrick McCabe a3f2414990
KAFKA-16624: Don't generate useless PartitionChangeRecord on older MV (#15810)
Fix a case where we could generate useless PartitionChangeRecords on metadata versions older than
3.6-IV0. This could happen in the case where we had an ISR with only one broker in it, and we were
trying to go down to a fully empty ISR. In this case, PartitionChangeBuilder would block the record
to going down to a fully empty ISR (since that is not valid in these pre-KIP-966 metadata
versions), but it would still emit the record, even though it had no effect.

Reviewers: Igor Soarez <soarez@apple.com>
2024-05-02 09:23:25 -07:00
Nikolay cdc4caa578
KAFKA-14588 UserScramCredentialsCommandTest rewritten in Java (#15832)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>
2024-05-02 10:35:10 +01:00
Kuan-Po (Cooper) Tseng 89d8045a15
KAFKA-16647 Remove setMetadataDirectory from BrokerNode/ControllerNode (#15833)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-02 09:04:15 +08:00
TaiJuWu d9c36299db
KAFKA-16614 Disallow @ClusterTemplate("") (#15800)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-02 07:15:22 +08:00
PoAn Yang 4825c89d14
KAFKA-16588 broker shutdown hangs when log.segment.delete.delay.ms is zero (#15773)
Instead of entering pending forever, this PR invoke next schedule after 1ms. However, the side effect is busy-waiting. Hence, This PR also update the docs to remind users about that - the issue about smaller log.segment.delete.delay.ms

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-01 17:11:20 +08:00
mannoopj 31355ef8f9
KAFKA-16475: add more tests to TopicImageNodeTest (#15735)
Add more test cases to TopicImageNodeTest.java.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-04-30 14:59:00 -07:00
Ken Huang da5f4424dc
MINOR: Clean up TestUtils.scala (#15808)
This PR do the following cleanup for TestUtils.scala

1) remove unused methods
2) move methods used by single test class out of

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-01 04:13:29 +08:00
Dongnuo Lyu 1e8415160f
MINOR: Add replayRecords to CoordinatorResult (#15818)
The patch adds a boolean attribute `replayRecords` that specifies whether the records should be replayed.

Reviewers: David Jacot <djacot@confluent.io>
2024-04-30 09:14:02 -07:00
Kirk True 7c0a302c4d
KAFKA-16555: Consumer's RequestState has incorrect logic to determine if inflight (#15723)
In some cases, the network layer is very fast and can process a response and send out a follow-up request within the same millisecond timestamp. This is causing problems due to the way we determine if we already have an inflight request.

The previous logic for tracking inflight status used timestamps: if the timestamp from the last received response was less than the timestamp from the last sent request, we'd interpret that as having an inflight request. However, this approach would incorrectly return false from RequestState.requestInFlight() if the two timestamps were equal.

One result of this faulty logic is that in such cases, the consumer would accidentally send multiple heartbeat requests to the consumer group coordinator. The consumer group coordinator would interpret these requests as 'join group' requests and create members for each request. Therefore, the coordinator was under the false understanding that there were more members in the group than there really were. Consequently, if your luck was really bad, the coordinator might assign partitions to one of the duplicate members. Those partitions would be assigned to a phantom consumer that was not reading any data, and this led to flaky tests.

This change introduces a stupid simple flag to RequestState that is set in onSendAttempt and cleared in onSuccessfulAttempt, onFailedAttempt, and reset. A new unit test has been added and this has been tested against all of the consumer unit and integration tests, and has removed all known occurrences of phantom consumer group members in the system tests.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Lianet Magrans <lianetmr@gmail.com>, Philip Nee <pnee@confluent.io>
2024-04-30 10:00:17 +02:00
PoAn Yang 9387c7a626
KAFKA-16467: add docs README (#15664)
Add how to run local website to docs folder.

Signed-off-by: PoAn Yang <payang@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-30 10:26:35 +08:00
Kuan-Po (Cooper) Tseng 6d436a8f98
KAFKA-16627 Remove ClusterConfig parameter in BeforeEach and AfterEach (#15824)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-30 08:40:28 +08:00
Johnny Hsu 78c7f08e20
MINOR: Reuse KafkaConfig to create MetadataLogConfig (#15788)
Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Kuan-Po (Cooper) Tseng <brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-30 08:04:16 +08:00
Johnny Hsu 150a78ab90
KAFKA-15897 fix kafka.server.ControllerRegistrationManagerTest#testWrongIncarnationId (#15828)
ControllerRegistrationManagerTest is flaky due to the poll in L221. The potential root cause is a race condition between the first poll (L221) and the second poll (L229). Before the second poll, we mock a response (L226), which should be processed by the second poll. However, if the first poll take this away, the second poll would get nothing, and this could lead to an error.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-30 07:55:12 +08:00
Nikolay 81c24d6bf8
KAFKA-15588 ConfigCommandIntegrationTest rewritten in java (#15645)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-30 01:46:08 +08:00
Omnia Ibrahim e1bfaec49d
KAFKA-15853 Move metrics configs out of KafkaConfig (#15822)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-30 01:19:05 +08:00
Lianet Magrans 636e65aa6b
KAFKA-16465: Fix consumer sys test revocation validation (#15778)
This fixes a consumer system test that was failing for the new protocol. The failure was because the test was expecting the eager behaviour of partitions being revoked on every rebalance, and it was wrongfully applying it to the runs with the new protocol too.
This same situation was previously identified and fixed in other parts of the sys test with #15661.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-04-29 11:56:36 +02:00
Luke Chen ec151c8278
KAFKA-16563: retry pollEvent in KRaftMigrationDriver for retriable errors (#15732)
When running ZK migrating to KRaft process, we encountered an issue that the migrating is hanging and the ZkMigrationState cannot move to MIGRATION state. And it is because the pollEvent didn't retry with the retriable MigrationClientException (ZK client retriable errors) while it should. This PR fixes it and add test. And because of this, the poll event will not poll anymore, which causes the KRaftMigrationDriver hanging.

Reviewers: Luke Chen <showuon@gmail.com>, Igor Soarez<soarez@apple.com>, Akhilesh C <akhileshchg@users.noreply.github.com>
2024-04-29 17:44:47 +08:00
Gantigmaa Selenge a9b4b88e54
MINOR: Remove unneccessary version from excluded dependencies of clients (#15804)
Small cleanup: removed version when excluding shaded dependencies from clients library as it's not needed.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-29 16:58:12 +08:00
Dongnuo Lyu 994077e43e
MINOR: Fix the flaky testConsumerGroupHeartbeatWithStableClassicGroup by sorting the topic partition list (#15816)
We are seeing flaky test in `testConsumerGroupHeartbeatWithStableClassicGroup` where the error is caused by the different ordering in the expected and actual values. The patch sorts the topic partition list in the records to fix the issue.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Igor Soarez <soarez@apple.com>, David Jacot <djacot@confluent.io>
2024-04-29 00:43:49 -07:00