Commit Graph

327 Commits

Author SHA1 Message Date
Xuan-Zhang Gong 2994e5eff3
KAFKA-19004 Move DelayedDeleteRecords to server-common module (#19226)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-04-03 00:06:27 +08:00
Sean Quah 7ee73e6cd5
MINOR: Accept Throwables in DeferredEventQueue.failAll (#19337)
DeferredEvents take a Throwable and not an Exception. For consistency,
DeferredEventQueue.failAll should also accept a Throwable.

Reviewers: David Jacot <djacot@confluent.io>
2025-04-01 05:25:13 -07:00
Dmitry Werner 84b8fec089
KAFKA-14486 Move LogCleanerManager to storage module (#19216)
Move LogCleanerManager and related classes to storage module and rewrite
in Java.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Jun Rao
<junrao@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping
Tsai <chia7712@gmail.com>
2025-03-27 12:35:38 +08:00
Sushant Mahajan eb88e78373
KAFKA-18827: Initialize share group state group coordinator impl. [3/N] (#19026)
* This PR adds impl for the initialize share groups call from the Group
Coordinator perspective.
* The initialize call on persister instance will be invoked by the
`GroupCoordinatorService`, based on the response of the
`GroupCoordinatorShard.shareGroupHeartbeat`. If there is new topic
subscription or member assignment change (topic paritions incremented),
the delta share partitions corresponding to the share group in question
are returned as an optional initialize request.
* The request is then sent to the share coordinator as an encapsulated
timer task because we want the heartbeat response to go asynchronously.
* Tests have been added for `GroupCoordinatorService` and
`GroupMetadataManager`. Existing tests have also been updated.
* A new formatter `ShareGroupStatePartitionMetadataFormatter` has been
added for debugging.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-26 19:40:23 +00:00
Sanskar Jhajharia 15474e0100
MINOR: Cleanup Server Common Module (#19085)
Given that now we support Java 17 on our brokers, this PR replace the
use of the following in server-common module:

- Collections.singletonList() and Collections.emptyList() with List.of()
- Collections.singletonMap() and Collections.emptyMap() with Map.of()
- Collections.singleton() and Collections.emptySet() with Set.of()
- Arrays.asList() with List.of()

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-25 01:14:00 +08:00
TengYao Chi 20bad6efb3
KAFKA-18576 Convert ConfigType to Enum (#18711)
JIRA: KAFKA-18576
After removing ZooKeeper, we no longer need to exclude `client_metrics`
and `group` from `ConfigType#ALL`.

Since it's a common pattern to provide a mechanism to know all values in
enumeration ( Java enum provides ootb), we should convert ConfigType to
enum.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-25 01:10:59 +08:00
PoAn Yang d497250c22
KAFKA-18999 Remove BrokerMetadata (#19227)
* Replace `BrokerMetadata` with `UsableBroker` in KRaftMetadataCache and
ReassignPartitionsCommand.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-22 19:30:28 +08:00
José Armando García Sancio d04efca493
KAFKA-18979; Report correct kraft.version in ApiVersions (#19205)
Skip kraft.version when applying FeatureLevelRecord records. The kraft.version is stored as control records and not as metadata records. This solution has the benefits of removing from snapshots any FeatureLevelRecord for kraft.version that was incorrectly written to the log and allows ApiVersions to report the correct finalized kraft.version.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-03-13 18:39:24 -04:00
Apoorv Mittal f3da8f500e
KAFKA-18936: Fix share fetch when records are larger than max bytes (#19145)
The PR fixes the behaviour when records are fetched which are larger
than `fetch.max.bytes` config.

The usage of `hardMaxBytesLimit` is in ReplicaManager where it decides
whether to fetch a single record or not. The file records get sliced
based on the bytes requested. However, if `hardMaxBytesLimit` is false
then at least one record is fetched and bytes are adjusted accordingly in
`localLog`.

Reviewers: Jun Rao <junrao@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Abhinav Dixit <adixit@confluent.io>
2025-03-12 09:03:35 +00:00
Lucas Brutschy fc2e3dfce9
MINOR: Disallow unused local variables (#18963)
Recently, we found a regression that could have been detected by static
analysis, since a local variable wasn't being passed to a method during
a refactoring, and was left unused. It was fixed in
[7a749b5](7a749b589f),
but almost slipped into 4.0. Unused variables are typically detected by
IDEs, but this is insufficient to prevent these kinds of bugs. This
change enables unused local variable detection in checkstyle for Kafka.

A few notes on the usage:
- There are two situations in which people actually want to have a local
variable but not use it. First, there are `for (Type ignored:
collection)` loops which have to loop `collection.length` number of
times, but that do not use `ignored` in the loop body. These are
typically still easier to read than a classical `for` loop. Second, some
IDEs detect it if a return value of a function such as `File.delete` is
not being used. In this case, people sometimes store the result in an
unused local variable to make ignoring the return value explicit and to
avoid the squiggly lines.
- In Java 22, unsued local variables can be omitted by using a single
underscore `_`. This is supported by checkstyle. In pre-22 versions,
IntelliJ allows such variables to be named `ignored` to suppress the
unused local variable warning. This pattern is often (but not
consistently) used in the Kafka codebase. This is, however, not
supported by checkstyle.

Since we cannot switch to Java 22, yet, and we want to use automated
detection using checkstyle, we have to resort to prefixing the unused
local variables with `@SuppressWarnings("UnusedLocalVariable")`. We have
to apply this in 11 cases across the Kafka codebase. While not being
pretty, I'd argue it's worth it to prevent bugs like the one fixed in
[7a749b5](7a749b589f).

Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur
<mumrah@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Bruno
Cadonna <cadonna@apache.org>, Kirk True <ktrue@confluent.io>
2025-03-10 09:37:35 +01:00
Colin Patrick McCabe 343bc995f4
KAFKA-18920: The kcontrollers must set kraft.version in ApiVersionsResponse (#19127)
The kafka controllers need to set kraft.version in their
ApiVersionsResponse messages according to the current kraft.version
reported by the Raft layer. Instead, currently they always set it to 0.

Also remove FeatureControlManager.latestFinalizedFeatures. It is not
needed and it does a lot of copying.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-07 13:46:46 -08:00
leaf-soba 53b2935c51
KAFKA-18461 Add Objects.requireNotNull to Snapshot (#18684)
Add explicit not-null checks in Snapshot so we get a better error message in the event that a Snapshot object is accessed after erase has been called.

Reviewers: David Arthur <mumrah@gmail.com>
2025-03-07 09:48:09 -05:00
co63oc 3d7ac0c3d1
MINOR: Fix typos in multiple files (#19102)
Fix typos in multiple files

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-05 14:27:32 +00:00
Logan Zhu 011f256c86
KAFKA-18886 add behavior change of CreateTopicPolicy and AlterConfigPolicy to zk2kraft (#19087)
1. Updated JavaDoc to reflect that CreateTopicPolicy and AlterConfigPolicy run on the controller in KRaft mode.
2. Modified Behavioral Change Reference in the HTML docs to include this change.
3. add warning message to KafkaConfig if the config of broker node has policy configs 


Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-05 15:15:03 +08:00
ClarkChen 269e2d898b
KAFKA-18849 Add "strict min ISR" to the docs of "min.insync.replicas" (#19016)
KIP-966 adds strict min ISR rule, so this PR improves the docs of min.insync.replicas to include that change.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-27 16:05:24 +08:00
Sushant Mahajan 6e76736890
KAFKA-18827: Initialize share group state persister impl [2/N]. (#18992)
* In this PR, we have provided implementation for the initialize share
group state RPC from the persister perspective.
* Tests have been added wherever applicable.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-23 08:03:13 +00:00
Calvin Liu 8f13e7c207
MINOR: Move the ELR default version to 4.1 (#18954)
- ELR is enabled (ELRV_1) by default if the cluster is created with its bootstrap metadata version >= IBP_4_1_IV0.
- ELRV_1 can be manually enabled iff the metadata version is >= IBP_4_0_IV1.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>, David Jacot <djacot@confluent.io>
2025-02-21 16:13:11 +01:00
Sushant Mahajan c2cb543a1e
KAFKA-18629: Delete share group state RPC group coordinator impl. [3/N] (#18848)
* In this PR, we have added GC side impl to call the delete state share
coord RPC using the persister.
* We will be using the existing `GroupCoordinatorService.deleteGroups`.
The logic will be modified as follows:
* After sanitization, we will call a new
`runtime.scheduleWriteOperation` (not read for consistency) with
callback `GroupCoordinatorShard.sharePartitions`. This will return a Map
of share partitions of the groups which are of SHARE type. We need to
pass all groups as WE CANNOT DETERMINE the type of the group in the
service class.
* Then using the map we will create requests which could be passed to
the persister and make the appropriate calls.
* Once this future completes, we will continue with the existing flow of
group deletion.
* If the group under inspection is not share group - the read callback
should return an empty map.
* Tests have been added wherever applicable.

Reviewers: David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-02-21 12:13:16 +00:00
Matthias J. Sax 538a60e1b3
MINOR: disallow rawtypes and fail build (#18877)
Cleanup code to avoid rawtype, and add suppressions where necessary.
Change the build to fail on rawtype warning.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-19 13:11:49 -08:00
Ismael Juma 3dba3125e9
KAFKA-18601: Assume a baseline of 3.3 for server protocol versions (#18845)
3.3.0 was the first KRaft release that was deemed production-ready and also
when KIP-778 (KRaft to KRaft upgrades) landed. Given that, it's reasonable
for 4.x to only support upgrades from 3.3.0 or newer (the metadata version also
needs to be set to "3.3" or newer before upgrading).

Noteworthy changes:
1. `AlterPartition` no longer includes topic names, which makes it possible to
simplify `AlterParitionManager` logic.
2. Metadata versions older than `IBP_3_3_IV3` have been removed and
`IBP_3_3_IV3` is now the minimum version.
3. `MINIMUM_BOOTSTRAP_VERSION` has been removed.
4. Removed `isLeaderRecoverySupported`, `isNoOpsRecordSupported`,
`isKRaftSupported`, `isBrokerRegistrationChangeRecordSupported` and
`isInControlledShutdownStateSupported` - these are always `true` now.
Also removed related conditional code.
5. Removed default metadata version or metadata version fallbacks in
multiple places - we now fail-fast instead of potentially using an incorrect
metadata version.
6. Update `MetadataBatchLoader.resetToImage` to set `hasSeenRecord`
based on whether image is empty - this was a previously existing issue that
became more apparent after the changes in this PR.
7. Remove `ibp` parameter from `BootstrapDirectory`
8. A number of tests were not useful anymore and have been removed.

I will update the upgrade notes via a separate PR as there are a few things that
need changing and it would be easier to do so that way.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Justine Olshan <jolshan@confluen.io>, Ken Huang <s7133700@gmail.com>
2025-02-19 05:35:42 -08:00
xijiu 4c4458c17a
KAFKA-18799 Remove AdminUtils (#18946)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-19 06:25:43 +08:00
Parker Chang ed366e6b89
MINOR: Align assertFutureThrows method signature with JUnit conventions (#18825)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-18 15:56:42 +00:00
Andrew Schofield 6c14f64245
MINOR: Rename NoOpShareStatePersister for consistency (#18933)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-18 14:07:59 +00:00
David Jacot aec0e555be
MINOR: Mark IBP_4_0_IV3 as production ready! (#18902)
This patch marks IBP_4_0_IV3 as production ready for the Apache Kafka 4.0 release. It also introduced IBP_4_1_IV0 as the next development version.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-02-14 08:17:11 -08:00
Calvin Liu e7a2af8414
KAFKA-18634: Fix ELR metadata version issues (#18680)
This patch cleans up the places that should not use MV to determine ELR is enabled marks 4.0IV1 stable.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2025-02-13 23:40:31 -08:00
Sushant Mahajan 9e5503b04f
KAFKA-18745: Handle network related errors in persister. (#18824)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-07 15:59:06 +00:00
Ken Huang a3d9d881e1
KAFKA-18530 Remove ZooKeeperInternals (#18641)
Since zk has been removed in 4.0, config handlers no longer need to handle the "<default>" value. This PR streamlines the config update process by eliminating the unnecessary string checks for "<default>"

Reviewers: Christo Lolov <lolovc@amazon.com>, Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-06 17:48:17 +08:00
Sushant Mahajan 0bd1ff936f
KAFKA-18629: Add persister impl and tests for DeleteShareGroupState RPC. [2/N] (#18748)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-05 14:51:19 +00:00
Sanskar Jhajharia 7dbed2f6e8
[KAFKA-16720] AdminClient Support for ListShareGroupOffsets (2/2) (#18671)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan <smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-02-05 14:38:09 +00:00
TengYao Chi aac62a32d9
KAFKA-18698: Migrate suitable classes to records in server and server-common modules (#18783)
Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Christo Lolov <lolovc@amazon.com>
2025-02-05 10:00:11 +00:00
Ming-Yen Chung 9f78771a1f
KAFKA-18693 Remove PasswordEncoder (#18790)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-04 13:18:41 +08:00
Calvin Liu 6235a73622
KAFKA-16540: Clear ELRs when min.insync.replicas is changed. (#18148)
In order to maintain the integrity of replication, we need to clear the ELRs of affected partitions when min.insync.replicas is changed. This could happen at the topic level, or at a global level if the cluster level default is changed.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2025-01-24 10:57:33 -08:00
TengYao Chi fa2df3bca7
KAFKA-18559 Cleanup FinalizedFeatures (#18593)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-24 19:39:01 +08:00
Ken Huang d0bca67b27
KAFKA-18363: Remove ZooKeeper mentiosn in broker configs (#18346)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, TengYao Chi <kitingiao@gmail.com>
2025-01-20 18:22:40 +01:00
TengYao Chi a842c02b88
KAFKA-18553: Update javadoc and comments of ConfigType (#18567)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <amittal@confluent.io>
2025-01-20 15:20:36 +01:00
Sanskar Jhajharia bcbc72e29b
[KAFKA-16720] AdminClient Support for ListShareGroupOffsets (1/n) (#18571)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-01-20 07:47:14 +00:00
TengYao Chi 3490cf7bfc
MINOR: Cleanup MetadataVersion (#18623)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-19 23:48:37 +08:00
Ismael Juma b90ef861dd
Convert BrokerEndPoint to record (#18577)
Tests for this class is being removed via #18573 - making it
a record avoids the need for the tests being removed.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-17 07:08:43 -08:00
Jason Taylor 23c459286b
KAFKA-16368: Update defaults for LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_DEFAULT and NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG (#18106)
Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-16 15:30:00 +01:00
Apoorv Mittal a1f7457389
KAFKA-18514: Refactor share module code to server and server-common (#18524)
As per the discussion with @ijuma and @mumrah, the `share` module seems not required and it's advised to user `server` and `server-common` instead. The PR moves the classes from `share` module to respective server related modules.

Following has been refactored in the PR:

- Moved Share Fetch, Acknowledge, Session, Context and Cache related classes to `server` module as the classes are used by `core` and `tools` modules.
- Moved `Persister` releated classes from `share` to `server-common` as the Persister classes though currently just being used by `core` module but in [near future](https://github.com/apache/kafka/pull/17775) will also be used by `group-coordinator`. Hence the Persister classes shouldn't go in `server`. The debate is mostly between `coordinator-common` vs `server-common`. We have kept the Persister in `server-common` for now, the classes are more related to the server than the coordinator. Persister is basically an abstraction in the server to let you choose how you want to persist the share group progress.
- Updated build.gradle to remove `share` module.
- Removed `import-control-share.xml`

Reviewers: Ismael Juma <ismael@juma.me.uk>
2025-01-16 00:14:25 -08:00
Xuan-Zhang Gong 1ee715473c
KAFKA-18446 Remove MetadataCacheControllerNodeProvider (#18437)
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-12 02:12:35 +08:00
Ismael Juma d4aee71e36
KAFKA-18465: Remove MetadataVersions older than 3.0-IV1 (#18468)
Apache Kafka 4.0 will only support KRaft and 3.0-IV1 is the minimum version supported by KRaft. So, we can assume that Apache Kafka 4.0 will only communicate with brokers that are 3.0-IV1 or newer.

Note that KRaft was only marked as production-ready in 3.3, so we could go further and set the baseline to 3.3. I think we should have that discussion, but it made sense to start with the non controversial parts.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <david.jacot@gmail.com>
2025-01-11 09:42:39 -08:00
mingdaoy 23e77ed2d4
KAFKA-18374 Remove EncryptingPasswordEncoder, CipherParamsEncoder, GcmParamsEncoder, IvParamsEncoder, and the unused static variables in PasswordEncoder (#18355)
Reviewers: TengYao Chi <kitingiao@gmail.com>, TaiJuWu <tjwu1217@gmail.com>, Christo Lolov <lolovc@amazon.com>, PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-07 03:48:33 +08:00
Ismael Juma 409a43eff7
MINOR: Collection/Option usage simplification via methods introduced in Java 9 & 11 (#18305)
Relevant methods:
1. `List.of`, `Set.of`, `Map.of` and similar (introduced in Java 9)
2. Optional: `isEmpty` (introduced in Java 11), `stream` (introduced in Java 9).

Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
2025-01-03 16:13:39 -08:00
Ismael Juma d6f24d3665
Use `instanceof` pattern to avoid explicit cast (#18373)
This feature was introduced in Java 16.

Reviewers: David Arthur <mumrah@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-01-02 09:32:51 -08:00
TengYao Chi 3161115ada
KAFKA-18361 Remove PasswordEncoderConfigs (#18347)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-31 00:18:23 +08:00
PoAn Yang b4be178599
KAFKA-17393: Remove log.message.format.version/message.format.version (KIP-724) (#18267)
Based on [KIP-724](https://cwiki.apache.org/confluence/display/KAFKA/KIP-724%3A+Drop+support+for+message+formats+v0+and+v1), the `log.message.format.version` and `message.format.version` can be removed in 4.0.

These configs effectively a no-op with inter-broker protocol version 3.0 or higher
since Apache Kafka 3.0, so the impact should be minimal.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2024-12-21 15:35:15 -08:00
Justine Olshan 8bd3746e0c
KAFKA-17705: Add Transactions V2 system tests and mark as production ready (#18132)
Added transaction version 2 to some of the system tests. Also marking TV2 as production ready.

Also fixes the defaultVersion test. 

Reviewers: Jun Rao <jun@confluent.io>
2024-12-21 14:01:54 -08:00
Ismael Juma fe56fc98fa
KAFKA-18269: Remove deprecated protocol APIs support (KIP-896, KIP-724) (#18218)
Included in this change:
1. Remove deprecated protocol api versions from json files.
3. Remove fields that are no longer used from json files (affects ListOffsets, OffsetCommit, DescribeConfigs).
4. Remove record down-conversion support from KafkaApis.
5. No longer return `Errors.UNSUPPORTED_COMPRESSION_TYPE` on the fetch path[1].
6. Deprecate `TopicConfig. MESSAGE_DOWNCONVERSION_ENABLE_CONFIG` and made the relevant
configs (`message.downconversion.enable` and `log.message.downcoversion.enable`) no-ops since
down-conversion is no longer supported. It was an oversight not to deprecate this via KIP-724.
7. Fix `shouldRetainsBufferReference` to handle null request schemas for a given version.
8. Simplify producer logic since it only supports the v2 record format now.
9. Fix tests so they don't exercise protocol api versions that have been removed.
10. Add upgrade note.

Testing:
1. System tests have a lot of failures, but those tests fail for trunk too and I didn't see any issues specific to this change - it's hard to be sure given the number of failing tests, but let's not block on that given the other testing that has been done (see below).
3. Java producers and consumers with version 0.9-0.10.1 don't have api versions support and hence they fail in an ungraceful manner: the broker disconnects and the clients reconnect until the relevant timeout is triggered.
4. Same thing seems to happen for the console producer 0.10.2 although it's unclear why since api versions should be supported. I will look into this separately, it's unlikely to be related to this PR.
5. Console consumer 0.10.2 fails with the expected error and a reasonable message[2].
6. Console producer and consumer 0.11.0 works fine, newer versions should naturally also work fine.
7. kcat 1.5.0 (based on librdkafka 1.1.0) produce and consume fail with a reasonable message[3][4].
8. kcat 1.6.0-1.7.0 (based on librdkafka 1.5.0 and 1.7.0 respectively) consume fails with a reasonable message[5].
9. kcat 1.6.0-1.7.0 produce works fine.
10. kcat 1.7.1  (based on librdkafka 1.8.2) works fine for consumer and produce.
11. confluent-go-client (librdkafka based) 1.8.2 works fine for consumer and produce.
12. I will test more clients, but I don't think we need to block the PR on that.

Note that this also completes part of KIP-724: produce v2 and lower as well as fetch v3 and lower are no longer supported.

Future PRs will remove conditional code that is no longer needed (some of that has been done in KafkaApis,
but only what was required due to the schema changes). We can probably do that in master only as it does
not change behavior.

Note that I did not touch `ignorable` fields even though some of them could have been
changed. The reasoning is that this could result in incompatible changes for clients
that use new protocol versions without setting such fields _if_ we don't manually
validate their presence. I will file a JIRA ticket to look into this carefully for each
case (i.e. if we do validate their presence for the appropriate versions, we can
set them to ignorable=false in the json file).

[1] We would return this error if a fetch < v10 was used and the compression topic config was set
to zstd, but we would not do the same for the case where zstd was compressed at the producer
level (the most common case). Since there is no efficient way to do the check for the common
case, I made it consistent for both by having no checks.
[2] ```org.apache.kafka.common.errors.UnsupportedVersionException: The broker is too new to support JOIN_GROUP version 1```
[3]```METADATA|rdkafka#producer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[4]```METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[5] `ERROR: Topic test-topic [0] error: Failed to query logical offset END: Local: Required feature not supported by broker`

Reviewers: David Arthur <mumrah@gmail.com>
2024-12-20 19:52:00 -08:00
TengYao Chi b37b89c668
KAFKA-9366 Upgrade log4j to log4j2 (#17373)
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations

Co-authored-by: Lee Dongjin <dongjin@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:14:31 +08:00
Justine Olshan 3cf8745243
MINOR: Add clientTransactionVersion to AddPartitionsToTxn requests and persist the value across transitions (#18086)
We can better keep track of which transactions use TV_2 by storing this information in the clientTransactionVersion field and persisting it across state transitions. Also updated some logging and equality code to include this information.

Added a test to ensure version persists. There aren't many TV2 transitions that don't specify TV, but I did test the InitProducerId + epoch overflow case.

Reviewers: Artem Livshits <alivshits@confluent.io>, Jeff Kim <jeff.kim@confluent.io>
2024-12-10 12:59:01 -08:00
Mickael Maison 4630628701
KAFKA-18144 Move the storage exceptions out of the core module (#18021)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-07 22:43:00 +08:00
Dongnuo Lyu e30edb3eff
KAFKA-18052: Decouple the dependency of feature stable version to the metadata version (#17886)
Currently the validation of feature upgrade relies on the supported version range generated during registration. For a given feature, its max supported feature version in production is set to be the default version value (the latest feature version with bootstrap metadata value smaller or equal to the latest production metadata value).

This patch introduces a LATEST_PRODUCTION value independent from the metadata version to each feature so that the highest supported feature version can be customized by the feature owner.

The change only applies to dynamic feature upgrade. During formatting, we still use the default value associated the metadata version.

Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
2024-12-05 11:07:47 -08:00
HYUNSANG HAN (한현상, Travis) 700bdd5fee
KAFKA-17997 Remove deprecated config log.message.timestamp.difference.max.ms (#17928)
Reviewers: Divij Vaidya <diviv@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-29 03:15:46 +08:00
Ken Huang c32a49549d
MINOR: Remove duplicate valid value in document (#17947)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-27 21:07:36 +08:00
Calvin Liu 2b2b3cd355
KAFKA-18062: use feature version to enable ELR (#17867)
Replace the ELR static config with feature version.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-11-26 14:40:23 -08:00
Colin Patrick McCabe 5fba067aaa
KAFKA-18063: SnapshotRegistry should not leak memory (#17898)
SnapshotRegistry needs to have a reference to all snapshot data structures. However, this should
not be a strong reference, but a weak reference, so that these data structures can be garbage
collected as needed. This PR also adds a scrub mechanism so that we can eventually reclaim the
slots used by GC'ed Revertable objects in the SnapshotRegistry.revertables array.

Reviewers: David Jacot <david.jacot@gmail.com>
2024-11-21 13:27:53 -08:00
Abhinav Dixit aa7a3dbd30
KAFKA-18027: MINOR: Correct DelayedOperationPurgatory code around adding of an already completed operation (#17842)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
2024-11-20 09:26:44 -08:00
Abhinav Dixit 77cc8ffbab
KAFKA-17948: Potential issue during tryComplete and onComplete simultaneous calls to access global variables (#17739)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
2024-11-15 10:26:54 -08:00
Mickael Maison 5d2074e8f2
MINOR: Move StopPartition to server-common (#17704)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-10 03:13:12 +08:00
Mickael Maison 0049b967e5
KAFKA-17890: Move DelayedOperationPurgatory to server-common (#17636)
Reviewers: Jun Rao <jun@confluent.io>, Apoorv Mittal <amittal@confluent.io>
2024-11-08 09:55:09 +01:00
Linsiyuan9 af53758746
KAFKA-17814 Use `final` declaration to replace the suppression `this-escape` (#17613)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-03 15:00:02 +08:00
Apoorv Mittal ff116df015
KAFKA-17002: Integrated partition leader epoch for Persister APIs (KIP-932) (#16842)
The PR integrates leader epoch for partition while invoking Persister APIs. The write RPC is retried once on leader epoch failure.

Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>
2024-10-30 14:41:39 -07:00
Ken Huang 30b1bdfc74
KAFKA-17835 Move ProducerIdManager and RPCProducerIdManager to transaction-coordinator module (#17562)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-31 02:40:47 +08:00
Apoorv Mittal 25a3590dc2
KAFKA-17813: Moving broker endpoint class and common server connection id (#17519)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Kuan-Po Tseng <brandboat@gmail.com>, Jun Rao <junrao@gmail.com>
2024-10-22 11:58:28 -07:00
Peter Lee 57abfc2493
KAFKA-17838 Move the log4j.properties from resources/test to resources(#17551)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-20 02:00:32 +08:00
Eric Chang 6b28e81ba1
KAKFA-17173 move quota config params from KafkaConfig to QuotaConfig (#17505)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-19 18:01:06 +08:00
Abhinav Dixit cb3b03377d
KAFKA-17742: Move DelayedShareFetchPurgatory declaration to ReplicaManager (#17437)
Declare the delayed share fetch purgatory inside ReplicaManager along with the existing purgatories. 

Check the share fetch purgatory when a replica becomes the follower or a replica is deleted from a broker through ReplicaManager.

Perform a checkAndComplete for share fetch when HWM is updated.

Reviewers:  Andrew Schofield <aschofield@confluent.io>,  Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
2024-10-17 13:58:10 -07:00
Apoorv Mittal 5791397d03
KAFKA-17776 Capability for connection disconnect listeners in socket server (#17473)
As of now, there isn't any way that resources can be cleaned in Kafka when client disconnects the connections. Generally time bound clean up happens which keep the resources occupied.

This PR provides capability to attach listeners in Socket Server which can react to client connection disconnections.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-10-17 12:28:46 +08:00
Gaurav Narula b03fe66cfe
KAFKA-17759 Remove Utils.mkSet (#17460)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-11 21:20:43 +08:00
David Arthur 51482fa298
MINOR Consolidate junit-platform.properties (#17399)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-08 10:51:33 -04:00
Mickael Maison 5624bc7c7e
MINOR: Remove hamcrest from connect:runtime, raft and server-common (#17394)
Reviewers: David Arthur <mumrah@gmail.com>
2024-10-08 14:22:45 +02:00
Federico Valeri 8dbbf5ea76
KAFKA-14705: Move topic filters to tools module (#17387)
After MirrorMaker 1 removal, there are no other modules dependencies for these classes, so we can safely move them to tools module.

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-10-07 16:54:36 +02:00
Justine Olshan 49d7ea6c6a
KAFKA-16308 [3/N]: Introduce feature dependency validation to UpdateFeatures command (#16443)
This change includes:

1. Dependency checking when updating the feature (all request versions)
2. Returning top level error and no feature level errors if any feature failed to update and using this error for all the features in the response. (all request versions)
3. Returning only top level none error for v2 and beyond

Reviewers: Jun Rao <jun@confluent.io>
2024-10-01 14:21:38 -07:00
Apoorv Mittal 05366d2fa7
KAFKA-17626: Move common fetch related classes from storage to server-common (#17289)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2024-09-26 20:31:28 -04:00
Justine Olshan ede0c94aaa
KAFKA-14562 [1/2]: Implement epoch bump after every transaction (#16719)
Implement server side changes for epoch bump but keep EndTxn as an unstable API until the client side changes are implemented. EndTxnResponse will return the producer ID and epoch for the transaction. Introduces new tagged fields to the TransactionLogValue to persist the clientTransactionVersion, previousProducerId, and nextProducerId to the log so that the state can be reloaded. See KIP-890 for more details.

Small updates to naming of lastProducerId -> PreviousProducerId. Also cleans up the many TransactionMetadata constructors.

Reviewers: Artem Livshits <alivshits@confluent.io>, David Jacot <djacot@confluent.io>
2024-09-26 09:37:11 -07:00
Kamal Chandraprakash 560076ba9e
KAFKA-15859: Add timeout field to the ListOffsets request (#17112)
This is part-3 of the KIP-1075.

Added a `timeoutMs` field to the ListOffsets request. This timeout is applicable only for the topic/partitions that are enabled with remote storage. 

When the timeout is defined in the request, then we use it to define the delay timeout for `DelayedRemoteListOffsets` request. When the timeout is not defined (requests from older client), then we take the dynamic `remote.list.offsets.request.timeout.ms` server config as the timeout.

Consumer and Admin client behavior are different. Consumer retries the LIST_OFFSETS request in-case of an error but not the AdminClient. And, consumer timeouts the request, if the response exceeds `request.timeout.ms`, whereas, AdminClient timeouts the request when it exceeds the `default.api.timeout.ms`. 

To retain the same behavior, we are passing the `requestTimeoutMs` as timeout from the consumer and defaultApiTimeout / overwritten ListOffsetsOption timeout from the admin.

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2024-09-26 11:03:17 +05:30
Sushant Mahajan 67f966f348
KAFKA-17469: Moved share external interfaces to share module. (#17262)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, David Arthur <mumrah@gmail.com>
2024-09-24 12:08:01 -04:00
Dmitry Werner 005155ba5c
KAFKA-17406: Move ClientIdAndBroker to server-common module (#16967)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, David Arthur <mumrah@gmail.com>
2024-09-04 11:04:40 +02:00
Dmitry Werner 5fd7ce2ace
KAFKA-17414 Move RequestLocal to server-common module (#16986)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-04 16:12:20 +08:00
Dmitry Werner 6836fa259e
KAFKA-17320 Move SensorAccess to server-common module (#16864)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-01 23:41:28 +08:00
Mickael Maison 1841c07d4a
KAFKA-17449 Move Quota classes to server-common module (#17060)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-31 12:41:34 +08:00
David Jacot c977bfdd3c
KAFKA-17413; Re-introduce `group.version` feature flag (#17013)
This patch re-introduces the `group.version` feature flag and gates the new consumer rebalance protocol with it. The `group.version` feature flag is attached to the metadata version `4.0-IV0` and it is marked as production ready. This allows system tests to pick it up directly by default without requiring to set `unstable.feature.versions.enable` in all of them. This is fine because we don't plan to do any incompatible changes before 4.0.

Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-29 01:22:54 -07:00
Sushant Mahajan 1621f88f06
KAFKA-17367: Share coordinator infra classes [1/N] (#16921)
Introduce ShareCoordinator interface and related classes.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-26 10:09:47 -04:00
Ken Huang a3aa6372ea
KAFKA-17336 Add IT to make sure the production MV does not use unstable version of LIST_OFFSET (#16893)
- due to the server config UNSTABLE_API_VERSIONS_ENABLE_CONFIG is true, so we can't test the scenario of ListOffsetsRequest is unstable version. We want to test this case in this PR
- get the MV from metadataCache.metadataVersion() instead of config.interBrokerProtocolVersion since MV can be set dynamically.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-23 03:47:55 +08:00
Justine Olshan fd544024aa
Make TestFeatureVersion 2 always map to MetadataVersion.latestTesting (#16901)
After some discussion on: https://github.com/apache/kafka/pull/16841#discussion_r1714322320

We decided it is best for test version to always map to MetadataVersion.latestTesting. We should always have one unstable MV (either because there is a feature being worked on OR when we mark the latest version stable we create a new one).

Reviewers: Jun Rao <junrao@apache.org>
2024-08-16 17:29:08 -07:00
Dmitry Werner bb0f4d5e11
MINOR: Remove unused `FeatureLevelListener` (#16815)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-13 08:31:27 +08:00
Colin Patrick McCabe 132e0970fb
KAFKA-17018: update MetadataVersion for the Kafka release 3.9 (#16841)
- Mark 3.9-IV0 as stable. Metadata version 3.9-IV0 should return Fetch version 17.

- Move ELR to 4.0-IV0. Remove 3.9-IV1 since it's no longer needed.

- Create a new 4.0-IV1 MV for KIP-848.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Justine Olshan <jolshan@confluent.io>
2024-08-12 16:30:43 -07:00
Kuan-Po Tseng 5d115fea3b
KAFKA-17310 locking the offline dir can destroy the broker exceptionally (#16856)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-12 16:44:18 +08:00
DL1231 bbdf79e1b4
KAFKA-14511; Extend AlterIncrementalConfigs API to support group config (#15067)
This patch add resources to store and handle consumer group's config.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>, David Jacot <djacot@confluent.io>
2024-08-12 00:40:13 -07:00
Colin Patrick McCabe e1b2adea07
KAFKA-17190: AssignmentsManager gets stuck retrying on deleted topics (#16672)
In MetadataVersion 3.7-IV2 and above, the broker's AssignmentsManager sends an RPC to the
controller informing it about which directory we have chosen to place each new replica on.
Unfortunately, the code does not check to see if the topic still exists in the MetadataImage before
sending the RPC. It will also retry infinitely. Therefore, after a topic is created and deleted in
rapid succession, we can get stuck including the now-defunct replica in our subsequent
AssignReplicasToDirsRequests forever.

In order to prevent this problem, the AssignmentsManager should check if a topic still exists (and
is still present on the broker in question) before sending the RPC. In order to prevent log spam,
we should not log any error messages until several minutes have gone past without success.
Finally, rather than creating a new EventQueue event for each assignment request, we should simply
modify a shared data structure and schedule a deferred event to send the accumulated RPCs. This
will improve efficiency.

Reviewers: Igor Soarez <i@soarez.me>, Ron Dagostino <rndgstn@gmail.com>
2024-08-10 12:31:45 +01:00
Josep Prat 4e862c0903
KAFKA-15875: Stops leak Snapshot in public methods (#16807)
* KAFKA-15875: Stops leak Snapshot in public methods

The Snapshot class is package protected but it's returned in
several public methods in SnapshotRegistry.
To prevent this accidental leakage, these methods are made
package protected as well. For getOrCreateSnapshot a new
method called IdempotentCreateSnapshot is created that returns void.
* Make builer package protected, replace <br> with <p>

Reviewers: Greg Harris <greg.harris@aiven.io>
2024-08-08 20:05:47 +02:00
bboyleonp666 5fac905749
KAFKA-17222 Remove the subclass of KafkaMetricsGroup (#16752)
The method overrides of metricName in KafkaMetricsGroup are no longer required since there's a new constructor that implement this with the same behavior.

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-07 01:26:49 +08:00
Mickael Maison 7c5d339d07
KAFKA-17227: Refactor compression code to only load codecs when used (#16782)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Josep Prat <josep.prat@aiven.io>
2024-08-06 11:01:21 +02:00
José Armando García Sancio e49c8df1f7 KAFKA-16533; Update voter handling
Add support for handling the update voter RPC. The update voter RPC is used to automatically update
the voters supported kraft versions and available endpoints as the operator upgrades and
reconfigures the KRaft controllers.

The add voter RPC is handled as follow:

1. Check that the leader has fenced the previous leader(s) by checking that the HWM is known;
   otherwise, return the REQUEST_TIMED_OUT error.

2. Check that the cluster supports kraft.version 1; otherwise, return the UNSUPPORTED_VERSION error.

3. Check that there are no uncommitted voter changes, otherwise return the REQUEST_TIMED_OUT error.

4. Check that the updated voter still supports the currently finalized kraft.version; otherwise
   return the INVALID_REQUEST error.

5. Check that the updated voter is still listening on the default listener.

6. Append the updated VotersRecord to the log. The KRaft internal listener will read this
   uncommitted record from the log and update the voter in the set of voters.

7. Wait for the VotersRecord to commit using the majority of the voters. Return a REQUEST_TIMED_OUT
   error if it doesn't commit in time.

8. Send the UpdateVoter successful response to the voter.

This change also implements the ability for the leader to update its own entry in the voter
set when it becomes leader for an epoch. This is done by updating the voter set and writing a
control batch as the first batch in a new leader epoch.

Finally, fix a bug in KafkaAdminClient's handling of removeRaftVoterResponse where we tried to cast
the response to the wrong type.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2024-08-05 11:32:21 -07:00
Colin Patrick McCabe 02f541d4ea
KAFKA-16518: Implement KIP-853 flags for storage-tool.sh (#16669)
As part of KIP-853, storage-tool.sh now has two new flags: --standalone, and --initial-voters. This PR implements these two flags in storage-tool.sh.

There are currently two valid ways to format a cluster:

The pre-KIP-853 way, where you use a statically configured controller quorum. In this case, neither --standalone nor --initial-voters may be specified, and kraft.version must be set to 0.

The KIP-853 way, where one of --standalone and --initial-voters must be specified with the initial value of the dynamic controller quorum. In this case, kraft.version must be set to 1.

This PR moves the formatting logic out of StorageTool.scala and into Formatter.java. The tool file was never intended to get so huge, or to implement complex logic like generating metadata records. Those things should be done by code in the metadata or raft gradle modules. This is also useful for junit tests, which often need to do formatting. (The 'info' and 'random-uuid' commands remain in StorageTool.scala, for now.)

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-08-02 15:47:45 -07:00
PoAn Yang 010ab19b72
KAFKA-16972 Move BrokerTopicMetrics to org.apache.kafka.storage.log.metrics (#16387)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-31 03:07:09 +08:00
Chung, Ming-Yen 7c0a96d08d
KAFKA-17185 Declare Loggers as static to prevent multiple logger instances (#16680)
As discussed in #16657 (comment) , we should make logger as static to avoid creating multiple logger instances.
I use the regex private.*Logger.*LoggerFactory to search and check all the results if certain logs need to be static.

There are some exceptions that loggers don't need to be static:
1) The logger in the inner class. Since java8 doesn't support static field in the inner class.
        https://github.com/apache/kafka/blob/trunk/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java#L3676

2) Custom loggers for each instance (non-static + non-final). In this case, multiple logger instances is actually really needed.
        https://github.com/apache/kafka/blob/trunk/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java#L166

3) The logger is initialized in constructor by LogContext. Many non-static but with final modifier loggers are in this category, that's why I use .*LoggerFactory to only check the loggers that are assigned initial value when declaration.
    
4) protected final Logger log = Logger.getLogger(getClass())
    This is for subclass can do logging with subclass name instead of superclass name.
    But in this case, if the log access modifier is private, the purpose cannot be achieved since subclass cannot access the log defined in superclass. So if access modifier is private, we can replace getClass() with <className>.class

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-31 02:37:36 +08:00
Justine Olshan a0f6e6f816
KAFKA-16192: Introduce transaction.version and usage of flexible records to coordinators (#16183)
This change includes adding transaction.version (part of KIP-1022)

New transaction version 1 is introduced to support writing flexible fields in transaction state log messages.

Transaction version 2 is created in anticipation for further KIP-890 changes.

Neither are made production ready. Tests for the new transaction version and new MV are created.

Also include change to not report a feature as supported if the range is 0-0.

Reviewers: Jun Rao <junrao@apache.org>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2024-07-26 11:38:44 -07:00
Kuan-Po Tseng 437b86cd20
KAFKA-17176 Move FileLock to server-common module (#16671)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-07-25 14:24:27 +08:00