Commit Graph

12596 Commits

Author SHA1 Message Date
Kuan-Po (Cooper) Tseng f895ab5145
KAFKA-16455: Check partition exists before send reassignments to server in ReassignPartitionsCommand (#15659)
Currently, when executing kafka-reassign-partitions.sh with the --execute option, if a partition number specified in the JSON file does not exist, this check occurs only when submitting the reassignments to alterPartitionReassignments on the server-side.

We can perform this check in advance before submitting the reassignments to the server side.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-09 07:56:31 +08:00
Kuan-Po (Cooper) Tseng 169ed60fe1
KAFKA-16477 Detect thread leaked client-metrics-reaper in tests (#15668)
After profiling the kafka tests, tons of client-metrics-reaper thread not cleanup after BrokerServer shutdown.
The thread client-metrics-reaper comes from ClientMetricsManager#expirationTimer, and BrokerServer#shudown doesn't close ClientMetricsManager which let the thread still runs in background.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-09 05:07:33 +08:00
vamossagar12 22f9eee89b
KAFKA-16481: Fixing flaky test kafka.server.ReplicaManagerTest#testRemoteLogReaderMetrics (#15677)
Adding NPE check in RLM.

Reviewers: Luke Chen <showuon@gmail.com>
2024-04-08 20:15:54 +08:00
Philip Nee 4e0578fb81
KAFKA-16156: beginningOrEndOffsets does not need to build an OffsetAndTimestamps object upon completion (#15525)
A subtle difference in the behavior of the two API causes the failures with Invalid negative timestamp.

In this PR, the list offsets response will be processed differently based on the API. For beginingOffsets/endOffsets - the offset response should be directly returned.

For offsetsForTimes - A OffsetAndTimestamp object is constructed for each requested TopicPartition before being returned.

The reason beginningOffsets and endOffsets - We are expecting a -1 timestamp from the response which subsequently causes the invalid timestamp exception because the original code tries to construct an OffsetAndTimestamp object upon returning.

In this PR, the following missing tasks are added:

short-circuit both beginningOrEndOffsets
Test both API (beginningOrEndOffsets, OffsetsForTime)
Seems like we don't have tests for this API: Note it is presented in other IntegrationTests but they are added to test Async consumer

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Lianet Magrans <lianetmr@gmail.com>
2024-04-08 13:04:58 +02:00
Erik van Oosten 8e61f04228
MINOR: Fix usage of none in javadoc (#15674)
- Use `Empty` instead of 'none' when referring to `Optional` values.
- `Headers.lastHeader` returns `null` when no header is found.
- Fix minor spelling mistakes.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-08 08:43:05 +08:00
Nikolay 4b2278ff9c
MINOR: Default test name added to core (#15667)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-07 21:43:16 +08:00
Nikolay 31e8a7fb04
MINOR: Default test name added to tools (#15666)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-07 21:35:00 +08:00
Calvin Liu 6de58d2731
MINOR; Missing minISR config should log a debug message (#15529)
Log a debug message when the min isr configuration is missing for a topic.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-04-06 17:41:32 -07:00
Manikumar Reddy dc189a39cc
MINOR: Improvements to release.py (#15651)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-06 20:55:25 +05:30
Gaurav Narula bdd85405e3
KAFKA-16293: Test log directory failure in Kraft (#15409)
Enables log directory failure system test for all Kraft modes in addition to ZK mode.

Reviewers: Luke Chen <showuon@gmail.com>, Igor Soarez <soarez@apple.com>, Proven Provenzano <pprovenzano@confluent.io>
2024-04-06 16:01:25 +08:00
Omnia Ibrahim e798bed198
KAFKA-16234: Log directory failure re-creates partitions in another logdir automatically (#15335)
This pr fixes the bug created by #15263 which caused topic partition to be recreated whenever the original log dir is offline: Log directory failure re-creates partitions in another logdir automatically

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>, Gaurav Narula <gaurav_narula2@apple.com>, Proven Provenzano <pprovenzano@confluent.io>
2024-04-06 14:36:26 +08:00
PoAn Yang 5c1b819b02
KAFKA-16472 Fix integration tests in Java with parameter name (#15663)
Following test cases don't really run kraft case. The reason is that the test info doesn't contain parameter name, so it always returns false in TestInfoUtils#isKRaft.

1) TopicCommandIntegrationTest
2) DeleteConsumerGroupsTest
3) AuthorizerIntegrationTest
4) DeleteOffsetsConsumerGroupCommandIntegrationTest

We can fix it by adding options.compilerArgs << '-parameters' after

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-06 10:43:22 +08:00
Manikumar Reddy fd9c7d2932
MINOR: Add 3.6.2 to system tests (#15665)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-05 19:36:23 +05:30
Andras Katona 5e4c7dae22
KAFKA-15915: Flaky ProducerIdManagerTest error injection fix (#15605)
testUnrecoverableErrors was flaky as the wanted error either affected the next block request (prefecthing) or just missed that.

First I tried to wait for the background thread to be finished before setting the Errors.X. But then it consistently failed, because the generateProducerId call does prefetching too and after a successful producer id generation we set the error and expected that it will fail again with coordinator-load-in-progress exception but since the block was prefetched, it was able to serve us with a proper producer id.

    calling generateProducerId --> no current block exists, so requesting block --> CoordinatorLoadInProgressException
    asserting exception
    calling generateProducerId again --> prefetching, requesting the next block --> giving back the producer id from the first block
    asserting received producer id
    setting error -- waiting for the background callback(s) to be finished first
    calling generateProducerId, expecting CoordinatorLoadInProgressException, but --> works like 2), just the prefetching callback is failing due to the error we set before

Note: without the waiting for the background thread completions the error setting could happened before the 2) step's callback or after that, the test was written in a way that it expected to happen before the cb.

This was the point I realised that we need to have a queue to control the responses rather than trying to do it in the middle of the test method.

Errors can be passed in a queue at creation of the mock id manager instead modifying on-the-fly.
In the queue we're specifying Errors, how the background thread (which imitates the controllerChannel) should behave, return an error or a proper response and call the callback accordingly with that.

I was able to simplify the mock manager id class as well, no need for the maybeRequestNextBlock overriding if the errors are handled this way via a queue.

Reviewers: Igor Soarez <soarez@apple.com>, Daniel Urban <urb.daniel7@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2024-04-05 14:59:27 +02:00
Johnny Hsu 16019b0e84
MINOR: remove redundant check in appendLegacyRecord (#15638)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-05 19:20:25 +08:00
Gaurav Narula a2ee0855ee
KAFKA-16471 invoke SSLEngine::closeInbound on SslTransportLayer close (#15655)
Invokes `SSLEngine::closeInbound` after we flush close_notify alert tothe socket. This fixes memory leak in Netty/OpenSSL based SSLEngine which only free native resources once closeInbound has been invoked.

Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-05 12:11:03 +08:00
Kuan-Po (Cooper) Tseng 862c2bfaf9
MINOR: remove redundnat list initialization (#15641)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-05 12:09:33 +08:00
Calvin Liu 376e9e20db
KAFKA-15586: Clean shutdown detection - server side (#14706)
If the broker registers with the same broker epoch as the previous session, it is recognized as a clean shutdown. Otherwise, it is an unclean shutdown. This replica will be removed from any ELR.

Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>
2024-04-04 09:12:05 -04:00
Apoorv Mittal ef7f823136
KAFKA-16359: Corrected manifest file for kafka-clients (#15532)
The issue KAFKA-16359 reported inclusion of kafka-clients runtime dependencies in MANIFEST.MF Class-Path.

The root cause is the issue defined here with the usage of shadow plugin.

Looking into the specifics of plugin and documentation, specifies that any dependency marked as shadow will be treated as following by the shadow plugin:

    1. Adds the dependency as runtime dependency in resultant pom.xml - code here
    2. Adds the dependency as Class-Path in MANIFEST.MF as well - code here

Resolution

We do need the runtime dependencies available in the pom (1 above) but not on manifest (2 above). Also there is no clear way to separate the behaviour as both above tasks relies on shadow configuration.

To fix, I have defined another custom configuration named shadowed which is later used to populate the correct pom (the code is similar to what shadow plugin does to populate pom for runtime dependencies).

Though this might seem like a workaround, but I think that's the only way to fix the issue. I have checked other SDKs which suffered with same issue and went with similar route to populate pom.

Reviewers: Luke Chen <showuon@gmail.com>, Reviewers: Mickael Maison <mickael.maison@gmail.com>, Gaurav Narula <gaurav_narula2@apple.com>
2024-04-04 17:56:08 +08:00
Kirk True c7ef80bb6c
KAFKA-16439: Update replication_replica_failure_test.py to support KIP-848’s group protocol config (#15629)
Added a new optional group_protocol parameter to the test methods, then passed that down to the setup_consumer method.

Unfortunately, because the new consumer can only be used with the new coordinator, this required a new @matrix block instead of adding the group_protocol=["classic", "consumer"] to the existing blocks 😢

Reviewers: Walker Carlson <wcarlson@apache.org>
2024-04-03 12:13:26 -05:00
Kirk True 6bb9caced0
KAFKA-16440: Update security_test.py to support KIP-848’s group protocol config (#15628)
Added a new optional group_protocol parameter to the test methods, then passed that down to the setup_consumer method.

Unfortunately, because the new consumer can only be used with the new coordinator, this required a new @matrix block instead of adding the group_protocol=["classic", "consumer"] to the existing blocks 😢

Reviewers: Walker Carlson <wcarlson@apache.org>
2024-04-03 12:13:14 -05:00
Kirk True 6569a354e6
KAFKA-16438: Update consumer_test.py’s static tests to support KIP-848’s group protocol config (#15627)
Migrated the following tests for the new consumer:

- test_fencing_static_consumer
- test_static_consumer_bounce
- test_static_consumer_persisted_after_rejoin

Reviewers: Walker Carlson <wcarlson@apache.org>
2024-04-03 12:13:03 -05:00
Kirk True e95e91a062
KAFKA-16275: Update kraft_upgrade_test.py to support KIP-848’s group protocol config (#15626)
Added a new optional group_protocol parameter to the test methods, then passed that down to the methods involved.

Unfortunately, because the new consumer can only be used with the new coordinator, this required a new @matrix block instead of adding the group_protocol=["classic", "consumer"] to the existing blocks 😢

Reviewers: Walker Carlson <wcarlson@apache.org>
2024-04-03 12:12:51 -05:00
PoAn Yang 21479a31bd
KAFKA-16413 add FileLockTest (#15624)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-03 21:13:45 +08:00
Greg Harris 3208c5f487
MINOR: AbstractConfig cleanup Part 2 (#15639)
Reviewers:  Manikumar Reddy <anikumar.reddy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-03 20:05:16 +08:00
rykovsi acecd370cc
KAFKA-16457 Useless import class `org.apache.kafka.common.config.ConfigDef.Type` (#15646)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-03 19:14:47 +08:00
Victor van den Hoven ee61bb721e
KAFKA-15417: move outerJoinBreak-flags out of the loop (#15510)
Follow up PR for https://github.com/apache/kafka/pull/14426 to fix a bug introduced by the previous PR.

Cf https://github.com/apache/kafka/pull/14426#discussion_r1518681146

Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-04-02 06:46:54 -07:00
Jeff Kim b3116f4f76
KAFKA-16148: Implement GroupMetadataManager#onUnloaded (#15446)
This patch completes all awaiting futures when a group is unloaded.

Reviewers: David Jacot <djacot@confluent.io>
2024-04-02 03:16:02 -07:00
Cheng-Kai, Zhang 7a10f4a17e
MINOR: enhance kafka-reassign-partitions command output (#15610)
Currently, when we using kafka-reassign-partitions to move the log directory, the output only indicates which replica's movement has successfully started.

This PR propose to show more detailed information, helping end users understand that the operation is proceeding as expected.

Reviewers: Luke Chen <showuon@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2024-04-02 14:54:01 +08:00
Igor Soarez 2932eb2b4c
KAFKA-16365: AssignmentsManager callback handling issues (#15521)
When moving replicas between directories in the same broker, future replica promotion hinges on acknowledgment from the controller of a change in the directory assignment.

ReplicaAlterLogDirsThread relies on AssignmentsManager for a completion notification of the directory assignment change.

In its current form, under certain assignment scheduling, AssignmentsManager both miss completion notifications, or prematurely trigger them.

Reviewers: Luke Chen <showuon@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Gaurav Narula <gaurav_narula2@apple.com>
2024-04-02 11:11:32 +08:00
Kamal Chandraprakash 2f733ac583
KAFKA-16161: Avoid empty remote metadata snapshot file in partition dir (#15636)
Avoid empty remote metadata snapshot file in partition dir

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Satish Duggana <satishd@apache.org>
2024-04-02 10:07:54 +08:00
Kuan-Po (Cooper) Tseng cc6b919212
KAFKA-16435 Add test for KAFKA-16428 (#15635)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-02 04:21:10 +08:00
Ori Hoch a524b6217b
MINOR: Fix doc of Admin#createDelegationToken (#15632)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-01 18:55:33 +08:00
Gaurav Narula 40e87ae35b
KAFKA-15823: disconnect from controller on AuthenticationException (#14760)
This PR changes the handling of authenticationException on a request from the node to the controller.

We disconnect controller connection and invalidate the cache so that the next run of the thread will establish a connection with the (potentially) updated controller.

Reviewers: Luke Chen <showuon@gmail.com>, Igor Soarez <soarez@apple.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
2024-04-01 11:55:08 +08:00
Johnny Hsu a640a81040
KAFKA-16323: fix testRemoteFetchExpiresPerSecMetric (#15463)
The variable of metrics item (kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec) is singleton object and it could be removed by other tests which are running in same JVM (and it is not recreated). Hence, verifying the metrics value is not stable to this test case.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2024-04-01 10:52:53 +08:00
Kuan-Po (Cooper) Tseng d4caa1c10e
MINOR: Remove redundant ApiVersionsResponse#filterApis (#15611)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-31 11:22:16 +08:00
PoAn Yang 9a9b532d5d
KAFKA-16447: Fix failed ReplicaManagerTest (#15630)
The change in https://github.com/apache/kafka/pull/15373/files#r1544335647 updated broker port from 9093 to 9094. Some of test cases check broker endpoint with fixed string 9093. I change test cases to get endpoint by broker id, so these cases will not fail if someone change the port again in the future.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-30 19:18:41 +08:00
Nikolay d8673b26bf
KAFKA-15899 [1/2] Move kafka.security package from core to server module (#15572)
1) This PR moves kafka.security classes from core to server module.
2) AclAuthorizer not moved, because it has heavy dependencies on core classes that not rewrited from scala at the moment.
3) AclAuthorizer will be deleted as part of ZK removal

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-30 11:54:22 +08:00
Calvin Liu 6e4a098055
KAFKA-16217: Stop the abort transaction try loop when closing producers (#15541)
This is a mitigation fix for the https://issues.apache.org/jira/browse/KAFKA-16217. Exceptions should not block closing the producers.
This PR reverts a part of the change #13591

Reviewers: Kirk True <ktrue@confluent.io>, Justine Olshan <jolshan@confluent.io>
2024-03-29 16:54:55 -07:00
Hector Geraldino 2118d85805
KAFKA-16223 Replace EasyMock/PowerMock with Mockito for KafkaConfigBackingStoreTest (1/3) (#15520)
Reviewers: Greg Harris <greg.harris@aiven.io>
2024-03-29 15:39:36 -07:00
Greg Harris 7317159f0c
KAFKA-16303: Add upgrade notes to 3.5.0, 3.5.2, and 3.7.0 about MM2 offset translation (#15423)
Signed-off-by: Greg Harris <greg.harris@aiven.io>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-03-29 08:51:42 -07:00
John Yu 7c3a596688
KAFKA-16397 Use ByteBufferOutputStream to avoid array copy (#15589)
Reviewers: Apoorv Mittal <amittal@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-03-29 20:28:19 +08:00
Greg Harris bf5e04e416
KAFKA-16349: Prevent race conditions in Exit class from stopping test JVM (#15484)
Signed-off-by: Greg Harris <greg.harris@aiven.io>
Reviewers: Chris Egerton <chrise@aiven.io>
2024-03-28 20:07:42 -07:00
Greg Harris 71bcac3b6a
MINOR: AbstractConfig cleanup (#15597)
Signed-off-by: Greg Harris <greg.harris@aiven.io>

Reviewers: Chris Egerton <chrise@aiven.io>, Mickael Maison <mickael.maison@gmail.com>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Matthias J. Sax <matthias@confluent.io>
2024-03-28 13:27:41 -07:00
Walker Carlson 8b274d8c1b
KAFKA-7663: Reprocessing on user added global stores restore (#15414)
When custom processors are added via StreamBuilder#addGlobalStore they will now reprocess all records through the custom transformer instead of loading directly.

We do this so that users that transform the records will not get improperly formatted records down stream.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-03-28 10:30:18 -05:00
Nikolay 355873aa54
MINOR: Use CONFIG suffix in ZkConfigs (#15614)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
Co-authored-by: n.izhikov <n.izhikov@vk.team>
2024-03-28 15:52:34 +01:00
Lianet Magrans b411ac0c70
KAFKA-16406 [2] : Split consumer commit tests (#15612)
Follow-up to #15535, splitting consumer integration tests defined in the long-running PlainTextConsumerTest. This PR extracts the tests that directly relate to committing offsets. No changes in logic.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2024-03-28 14:15:56 +01:00
Alyssa Huang 4ccbf1634a
MINOR: Metadata image test improvements (#15373)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-03-28 11:22:02 +01:00
Kuan-Po (Cooper) Tseng 4cb6806cb8
KAFKA-16232: kafka hangs forever in the starting process if the authorizer future is not returned (#15549)
add logs before and after future waiting, to allow admin to know we're waiting for the authorizer future.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-28 10:31:22 +08:00
Colin Patrick McCabe f40c06690b
KAFKA-16428: Fix bug where config change notification znode may not get created during migration (#15608)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Luke Chen <showuon@gmail.com>
2024-03-27 12:26:36 -07:00