Commit Graph

281 Commits

Author SHA1 Message Date
Alyssa Huang 7339d65b27
KAFKA-19354: KRaft observer should send fetch to best node (#19854)
Observers may get stuck fetching from bootstrap servers even on
discovery of a leader from a fetch response.

Observers currently fetch from bootstrap servers once their fetch
timeout expires, and even on rediscovery of the leader (e.g. observer
receives fetch response indicating location of leader), observers will
not  attempt to resume fetching from the leader.

This change simplifies observer polling logic to just rely on
maybeSendFetchToBestNode which takes care of the logic of selecting
either the leader or bootstrap servers  as the destination based on
timeouts and backoffs.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2025-07-28 13:03:14 -04:00
Kevin Wu 93447d5b88
KAFKA-19400: Update AddRaftVoterRequest RPC to version 1 (#19982)
Add the ackWhenCommitted boolean field to the AddRaftVoter request, and
bump the RPC's version to 1.

The default value of the ackWhenCommitted field is true, and in this
case the leader will return a response after committing the VotersRecord
generated by the RPC. If ackWhenCommitted is  false, the leader will
return a response after generating the new VotersRecord and adding it to
the batch accumulator.

Reviewers: Alyssa Huang <ahuang@confluent.io>, José Armando García
 Sancio <jsancio@apache.org>
2025-07-24 10:38:31 -04:00
Federico Valeri f2cbc7e3f2
MINOR: KafkaRaftClient cleanup (#20161)
Various minor cleanups for the KRaft client.

---------

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>

Reviewers: Luke Chen <showuon@gmail.com>
2025-07-17 10:41:28 +08:00
José Armando García Sancio 742b327025
KAFKA-14145; Faster KRaft HWM replication (#19800)
CI / build (push) Waiting to run Details
This change compares the remote replica's HWM with the leader's HWM and
completes the FETCH request if the remote HWM is less than the leader's
HWM. When the leader's HWM is updated any pending FETCH RPC is
completed.

Reviewers: Alyssa Huang <ahuang@confluent.io>, David Arthur
 <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-06-17 13:00:43 -04:00
Jhen-Yung Hsu 2e968560e0
MINOR: Cleanup simplify set initialization with Set.of (#19925)
Simplify Set initialization and reduce the overhead of creating extra
collections.

The changes mostly include:
- new HashSet<>(List.of(...))
- new HashSet<>(Arrays.asList(...)) / new HashSet<>(asList(...))
- new HashSet<>(Collections.singletonList()) / new
HashSet<>(singletonList())
- new HashSet<>(Collections.emptyList())
- new HashSet<>(Set.of())

This change takes the following into account, and we will not change to
Set.of in these scenarios:
- Require `mutability` (UnsupportedOperationException).
- Allow `duplicate` elements (IllegalArgumentException).
- Allow `null` elements (NullPointerException).
- Depend on `Ordering`. `Set.of` does not guarantee order, so it could
make tests flaky or break public interfaces.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-11 18:36:14 +08:00
Ken Huang df73133f3b
MINOR: Follow up KAFKA-19080 MetadataLogConfig (#19842)
CI / build (push) Waiting to run Details
See Discussion:
https://github.com/apache/kafka/pull/19371#discussion_r2109549343

Do the following changes:
- Update the internal config name with metadata prefix
- add the warning message for setting
`INTERNAL_METADATA_LOG_SEGMENT_BYTES_CONFIG`

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-09 03:13:03 +08:00
Ken Huang bcda92b5b9
KAFKA-19080 The constraint on segment.ms is not enforced at topic level (#19371)
CI / build (push) Waiting to run Details
The main issue was that we forgot to set
`TopicConfig.SEGMENT_BYTES_CONFIG` to at least `1024 * 1024`, which
caused problems in tests with small segment sizes.

To address this, we introduced a new internal config:
`LogConfig.INTERNAL_SEGMENT_BYTES_CONFIG`, allowing us to set smaller
segment bytes specifically for testing purposes.

We also updated the logic so that if a user configures the topic-level
segment bytes without explicitly setting the internal config, the
internal value will no longer be returned to the user.

In addition, we removed
`MetadataLogConfig#METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG` and added
three new internal configurations:
- `INTERNAL_MAX_BATCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_MAX_FETCH_SIZE_IN_BYTES_CONFIG`
- `INTERNAL_DELETE_DELAY_MILLIS_CONFIG`

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-25 20:57:22 +08:00
Alyssa Huang 97db06689b
KAFKA-18345; Wait the entire election timeout on election loss (#19747)
CI / build (push) Waiting to run Details
Replaces exponential backoff for candidate state after losing election
with waiting rest of election timeout. There is no need to have an
exponential backoff when the election timeout already provides a natural
throttle and it is randomized.

Reviewers: José Armando García Sancio <jsancio@apache.org>, TaiJuWu
 <tjwu1217@gmail.com>
2025-05-23 12:35:49 -04:00
Alyssa Huang c28f46459a
KAFKA-18345; Prevent livelocked elections (#19658)
CI / build (push) Waiting to run Details
At the retry limit binaryExponentialElectionBackoffMs it becomes
statistically likely that the exponential backoff returned
electionBackoffMaxMs. This is an issue as multiple replicas can get
stuck starting elections at the same cadence.

This change fixes that by added a random jitter to the max election
backoff.

Reviewers: José Armando García Sancio <jsancio@apache.org>, TaiJuWu
 <tjwu1217@gmail.com>, Yung <yungyung7654321@gmail.com>
2025-05-12 16:23:18 -04:00
Nick Guo 707a44a6cb
KAFKA-19068 Eliminate the duplicate type check in creating ControlRecord (#19346)
CI / build (push) Waiting to run Details
jira: https://issues.apache.org/jira/browse/KAFKA-19068

`RecordsIterator#decodeControlRecord` do the type check and then
`ControlRecord` constructor does that again.

we should add a static method to ControlRecord to create `ControlRecord`
with type check, and then `ControlRecord` constructor should be changed
to private to ensure all instance is created by the static method.

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-05-11 00:07:00 +08:00
José Armando García Sancio 2df14b1190
MINOR; Log message for unexpected buffer allocation (#19596)
Log a message when reading a batch that is larger than the currently
allocated batch.

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, PoAn Yang
 <payang@apache.org>
2025-05-06 12:01:49 -04:00
YuChia Ma 979f49f967
KAFKA-19146 Merge OffsetAndEpoch from raft to server-common (#19475)
CI / build (push) Waiting to run Details
1. remove org.apache.kafka.raft.OffsetAndEpoch
2. rewrite org.apache.kafka.server.common.OffsetAndEpoch by record
keyword
3. rename OffsetAndEpoch#leaderEpoch to OffsetAndEpoch#epoch

Reviewers: PoAn Yang <payang@apache.org>, Xuan-Zhang Gong
 <gongxuanzhangmelt@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-02 03:12:52 +08:00
NICOLAS GUYOMAR 19609d5986
MINOR; Log remote replica id when rejecting vote (#19600)
Adding the replicaId in the id mismatch log message to be able to
troubleshoot which node sent the erroneous VOTE request.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2025-04-30 10:42:29 -04:00
José Armando García Sancio b97a130c08
KAFKA-16538; Enable upgrading kraft version for existing clusters (#19416)
This change implements upgrading the kraft version from 0 to 1 in existing clusters.
Previously, clusters were formatted with either version 0 or version 1, and could not
be moved between them.

The kraft version for the cluster metadata partition is recorded using the
KRaftVersion control record. If there is no KRaftVersion control record
the default kraft version is 0.

The kraft version is upgraded using the UpdateFeatures RPC. These RPCs
are handled by the QuorumController and FeatureControlManager. This
change adds special handling in the FeatureControlManager so that
upgrades to the kraft.version are directed to
RaftClient#upgradeKRaftVersion.

To allow the FeatureControlManager to call
RaftClient#upgradeKRaftVersion is a non-blocking fashion, the kraft
version upgrade uses optimistic locking. The call to
RaftClient#upgradeKRaftVersion does validations of the version change.
If the validations succeeds, it generates the necessary control records
and adds them to the BatchAccumulator.

Before the kraft version can be upgraded to version 1, all of the
brokers and controllers in the cluster need to support kraft version 1.
The check that all brokers support kraft version 1 is done by the
FeatureControlManager. The check that all of the controllers support
kraft version is done by KafkaRaftClient and LeaderState.

When the kraft version is 0, the kraft leader starts by assuming that
all voters do not support kraft version 1. The leader discovers which
voters support kraft version 1 through the UpdateRaftVoter RPC. The
KRaft leader handles UpdateRaftVoter RPCs by storing the updated
information in-memory until the kraft version is upgraded to version 1.
This state is stored in LeaderState and contains the latest directory
id, endpoints and supported kraft version for each voter.

Only when the KRaft leader has received an UpdateRaftVoter RPC from all
of the voters will it allow the upgrade from kraft.version 0 to 1.

Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
2025-04-22 16:02:51 -07:00
leaf-soba 6f5be29374
KAFKA-18332 fix ClassDataAbstractionCoupling problem in KafkaRaftClientTest(1/2) (#18926)
- extract a unit test named `KafkaRaftClientClusterAuthTest` to reduce
the number of imported class

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-04-17 22:41:37 +08:00
Logan Zhu 50fb993ce0
KAFKA-19136 Move metadata-related configs from KRaftConfigs to MetadataLogConfig (#19465)
Separates metadata-related configurations from the `KRaftConfigs` into
the `MetadataLogConfig` class.

Previously, metadata-related configs were placed in `KRaftConfigs`,
which mixed server-related configs (like process.roles) with
metadata-specific ones (like metadata.log.*), leading to confusion and
tight coupling.

In this PR:
- Extract metadata-related config definitions and variables from
`KRaftConfig` into `MetadataLogConfig`.
- Move `node.id` out of `MetadataLogConfig` into `KafkaMetadataLog’s
constructor` to avoid redundant config references.
- Leave server-related configurations in `KRaftConfig`, consistent with
its role.

This separation makes `KafkaConfig` and `KRaftConfig` cleaner, and
aligns with the goal of having a dedicated MetadataLogConfig class for
managing metadata-specific configurations.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-04-17 22:17:11 +08:00
TengYao Chi b649b1ed5d
KAFKA-18935: Ensure brokers do not return null records in FetchResponse (#19167)
JIRA: KAFKA-18935  This patch ensures the broker will not return null
records in FetchResponse.   For more details, please refer to the
ticket.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai
 <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
2025-04-10 22:21:00 +08:00
Ming-Yen Chung 8c77953d5f
MINOR: Revert "migrate LogFetchInfo, Assignment and RequestAndCompletionHandler to java record" (#19177)
Revert some java record migration in #19062  #18783 

We assume java record is purely immutable data carriers.
As discussed in
https://github.com/apache/kafka/pull/19062#issuecomment-2709637352, if a
class has fields that may be mutable, we shouldn't migrate it to Java
record because the hashcode/equals behavior are changed.

* LogFetchInfo (Records)
* Assignment (successCallback)
* Remove `equals` method from Assignment since `Assignment` is not and
shouldn't be used in Map/Set key.
* RequestAndCompletionHandler (handler)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-04-05 23:01:17 +08:00
Sanskar Jhajharia 75f384a019
MINOR: Cleanup Raft Module (#19284)
Given that now we support Java 17 on our brokers, this PR replace the
use of the following in raft 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: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-04-02 21:46:48 +08:00
Vikas Singh 56d1dc1b6e
MINOR: Use readable interface to parse requests (#19163)
The generated request data type's constructors take Readable as an input. However, the parse method in the
AbstractRequest takes a ByteBuffer as input. So to create the corresponding request data objects, each individual
concrete Request classes wraps the ByteBuffer into a ByteBufferAccessor.

This is boilerplate code present in all the concrete request classes. This changes AbstractRequest's parse method so that subclasses can simply pass the `Readable` they get directly to request data classes.

The same change is made to the serialize method to maintain symmetry.

Reviewers: Ismael Juma <ismael@juma.me.uk>, José Armando García Sancio
<jsancio@apache.org>, Artem Livshits <alivshits@confluent.io>,
Truc Nguyen <trnguyen@confluent.io>
2025-03-26 10:13:13 -04:00
Mickael Maison 121ec2a662
KAFKA-15599 Move MetadataLogConfig to raft module (#19246)
Rewrite the class in Java and move it to the raft module.

Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-21 13:44:20 +08:00
TaiJuWu a2653281c2
KAFKA-18837: Ensure controller quorum timeouts and backoffs are at least 0 (#18998)
Add validators to the QuorumConfig's configuration definitions.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ken Huang <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>
2025-03-12 16:43:38 +01: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
Ming-Yen Chung 119d043c49
KAFKA-18700 Migrate SnapshotPath, Entry, OffsetAndEpoch, LogFetchInfo, and LogAppendInfo to record classes (#19062)
Migrate the following data carrier class to records to eliminate
constructors, `equals`, `hashCode`, and `toString`.
* `Entry` in `LogHistory`
* `SnapshotPath`

Additionally, migrate the following classes as discussed:
* OffsetAndEpoch
* LogFetchInfo
* LogAppendInfo

In Java, accessing a field in record class requires parentheses.
In Scala, parentheses are not needed because Scala allows omitting them
when calling parameterless methods; hence, there is no need to change
the Scala code.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 09:52:02 +08:00
José Armando García Sancio 4a8a0637e0
KAFKA-18723; Better handle invalid records during replication (#18852)
For the KRaft implementation there is a race between the network thread,
which read bytes in the log segments, and the KRaft driver thread, which
truncates the log and appends records to the log. This race can cause
the network thread to send corrupted records or inconsistent records.
The corrupted records case is handle by catching and logging the
CorruptRecordException. The inconsistent records case is handle by only
appending record batches who's partition leader epoch is less than or
equal to the fetching replica's epoch and the epoch didn't change
between the request and response.

For the ISR implementation there is also a race between the network
thread and the replica fetcher thread, which truncates the log and
appends records to the log. This race can cause the network thread send
corrupted records or inconsistent records. The replica fetcher thread
already handles the corrupted record case. The inconsistent records case
is handle by only appending record batches who's partition leader epoch
is less than or equal to the leader epoch in the FETCH request.

Reviewers: Jun Rao <junrao@apache.org>, Alyssa Huang <ahuang@confluent.io>, Chia-Ping Tsai <chia7712@apache.org>
2025-02-25 20:09:19 -05: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
kevin-wu24 184b891871
KAFKA-16524; Metrics for KIP-853 (#18304)
This change implement some of the metrics enumerated in KIP-853.

The KafkaRaftMetrics object now exposes number-of-voters, number-of-observers and uncommitted-voter-change. The number-of-observers and uncommitted-voter-change metrics are only present on the active controller or leader, since it does not make sense for other replicas to report these metrics.

In order to make these two metrics thread-safe, KafkaRaftMetrics needs to be passed into LeaderState, and therefore QuorumState. This introduces a circularity since the KafkaRaftMetrics constructor takes in QuorumState. To break the circularity for now, the logic using QuorumState will be moved to the KafkaRaftMetrics#initialize method.

The BrokerServerMetrics object now exposes ignored-static-voters. The ControllerServerMetrics object now exposes IgnoredStaticVoters. To implement both metrics for "ignored static voters", this PR introduces the ExternalKRaftMetrics interface, which allows for higher layer metrics objects to be accessible within the raft module.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2025-01-30 18:35:01 -05:00
Ismael Juma ca5d2cf76d
KAFKA-18646: Null records in fetch response breaks librdkafka (#18726)
Ensure we always return empty records (including cases where an error is returned).
We also remove `nullable` from `records` since it is effectively expected to be
non-null by a large percentage of clients in the wild.

This behavior regressed in fe56fc9 (KAFKA-18269). Empty records were
previously set via `FetchResponse.recordsOrFail(partitionData)` in the
now-removed `maybeConvertFetchedData` method.

Added an integration test that fails without this fix and also update many
tests to set `records` to `empty` instead of leaving them as `null`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2025-01-29 07:04:12 -08:00
Alyssa Huang 4583b033f0
KAFKA-17642: PreVote response handling and ProspectiveState (#18240)
This PR implements the second part of KIP-996 and KAFKA-16164 (tasks KAFKA-16607, KAFKA-17642, KAFKA-17643, KAFKA-17675) which encompass the response handling of PreVotes, addition of new ProspectiveState, update to metrics, and addition of Raft simulation tests.

Voters now transition to ProspectiveState first before CandidateState to prevent unnecessary epoch bumps. Voters in ProspectiveState send PreVotes requests which are Vote requests with PreVote set to true.

Follower grants PreVotes if it has not yet fetched successfully from leader. Leader denies all PreVotes. Unattached, Prospective, Candidate, and Resigned will grant PreVotes if the requesting replica's log is at least as long as theirs. Granted PreVotes are not persisted like standard votes. It is possible for a voter to grant several PreVotes in the same epoch.

The only state which is allowed to transition directly to CandidateState is ProspectiveState. This happens on reception of majority of granted PreVotes or if at least one voter doesn't support PreVote requests.

Prospective will transition to Follower after election loss/timeout if it was already aware of last known leader and the leader's endpoint, or at any point if it discovers the leader.

Prospective will transition to Unattached after election loss/timeout if it does not know the leader endpoints.

After electionTimeout, Resigned now always transitions to Unattached and increases the epoch.

Prospective grants standard votes if it has not already granted a standard vote (no votedKey), has no leaderId, and the recipient's log is current enough

Candidate no longer backs off after election timeout. Candidate still backs off after election loss.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2025-01-17 09:38:03 -05:00
Hailey Ni 15e938ef43
KAFKA-17973: Relax Restriction for Voters Set Change (#17728)
Relax the voter set change validation that exists in KRaft. When reading the kraft partition and validating voter set changes allow the voter set to have more than one change.

This violates the invariant that after a voter change there are overlapping voters for all possible majorities. This is okay because the KRaft leader checks that there are no pending voter set updates when handling an add voter request and a remove voter request.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2025-01-13 11:44:46 -05:00
Mickael Maison d1aa370db8
KAFKA-15599: Move SegmentPosition/TimingWheelExpirationService to raft module (#18094)
Reviewers: Divij Vaidya <diviv@amazon.com>, Jason Taylor <jastaylr@amazon.com>
2025-01-08 12:49:38 +01:00
PoAn Yang a52aedd6ff
KAFKA-18388 test-kraft-server-start.sh should use log4j2.yaml (#18370)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-07 04:20:06 +08:00
kevin-wu24 9818e9db26
KAFKA-17278; Add KRaft RPC compatibility tests (#17801)
Adds RPC compatibility tests for FETCH and FETCH_SNAPSHOT.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2025-01-06 10:18:13 -05:00
TengYao Chi 2e4a378c27
KAFKA-18131: Improve logs for voters (#18028)
Currently, the log of LeaderState#timeUntilCheckQuorumExpires uses streams without a terminal operator, resulting in output like java.util.stream.ReferencePipeline$3@39660237.
This PR aims to fix this issue and improve the log message.

Reviewers: Luke Chen <showuon@gmail.com>
2025-01-06 18:12:46 +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 09647c9b95
MINOR: Fix broken javadoc in BatchAccumulator (#18348)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2024-12-31 11:03:20 +00: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
Alyssa Huang b73e31eb15
KAFKA-17641; Update Vote RPC with PreVote field (#17807)
Introduces v2 of Vote RPC and implements the handling of the new version of the RPC.

Many references to "candidate" in the Vote RPC are changed to the more generic "replica". Replicas sending Vote request with PreVote set to true are not candidate. They are instead prospective candidate that are attempting to become candidate.

Replicas receiving PreVote requests (vote request with PreVote=true) with an epoch equal to their own will _not_ transition to Unattached state. They will only grant the vote if they have not recently fetched from leader and the request's last epoch and offset are up-to-date with theirs.

If a replica receives a PreVote request with an epoch greater than their current epoch, they will transition to Unattached state (setting their epoch to the one from the pre-vote request) and then grant the vote if the request's last epoch and offset are up-to-date with theirs.

To avoid a possible ping-pong scenario. For example, there is 3 node quorum, leader node A disconnects from quorum, node B goes into prospective state first before node C, node B sends pre-vote request to node C still in follower state and receives back that node A is leader, node B transitions to follower while node C transitions to prospective after election timeout. If you repeat this interaction, it is possible for such replicas to transition from Follower to Prospective in perpetuity. This issue is resolved by having follower state nodes grant pre-vote requests only if they have successfully fetched from the leader at least once after becoming a follower.

This change introduces a new suite called KafkaRaftClientPreVoteTest, for additional KRaft protocol tests with respect to pre-vote.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-12-13 16:24:30 -05: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
Alyssa Huang e979fce94e
KAFKA-17030; Unattached voters will fetch from bootstrap servers (#17352)
Because the set of voters are dynamic (KIP-953), it is possible for a replica to believe they are a voter while the current leader doesn't have that replica in the voter set. In this replicated state, the leader will not sent BeginQuorumEpoch requests to such a replica. This means that such replicas will not be able to discover the leader.

This change will help Unattached rediscover the leader by sending Fetch requests to the the bootstrap servers.
Followers have a similar issue - if they are unable to communicate with the leader they should try contacting the bootstrap servers.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-12-11 11:38:14 -05:00
kevin-wu24 3a9777a667
KAFKA-14619; KRaft validate snapshot id are at batch boundaries (#17500)
When KafkaRaftClient receives a request to create a snapshot with end offset that is not aligned to a batch boundary, do not create a misaligned snapshot and instead log at info level and throw an IllegalArgumentException.

Checking that the end offset is at a batch boundary is performed by reading the log at snapshotId.offset() and checking whether the offset in question is the base offset of the returned batch.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2024-12-09 11:38:00 -05:00
yx9o 38e727fe4d
KAFKA-17864: add descriptions to fields in the agreement (#17681)
Improve descriptive information in Kafka protocol documentation.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2024-12-07 18:47:11 +00: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
Mickael Maison c0a092f562
MINOR: Cleanups in raft module (#17877)
Reviewers: Yash Mayya <yash.mayya@gmail.com>
2024-11-21 15:19:07 +01:00
Justin Lee a8f84cab95
KAFKA-18001: Support UpdateRaftVoterRequest in KafkaNetworkChannel (#17773)
Adds support for UpdateRaftVoterRequest in KafkaNetworkChannel. This addresses the following scenario:

* Bootstrap a KRaft Controller quorum in dynamic mode
* Start additional controllers (as observers)
* Update kraft.version feature from 0 to 1
* Use kafka-metadata-quorum add-controller to promote an observer controller to a follower

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, Alyssa Huang <ahuang@confluent.io>
2024-11-15 15:55:01 -05: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
Mahsa Seifikar dafa126223
MINOR: fix the class name of UnattachedStateWithVoteTest to match the file name (#17662)
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-02 13:46:21 +08:00
Kuan-Po Tseng 60ea0999fb
KAFKA-17728 Add missing config `replica-directory-id` to raft README (#17518)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-17 11:32:27 +08:00
Logan Zhu d55b4e640c
KAFKA-17763 Remove Utils.covariantCast and use var for type inference (#17468)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-14 15:22:43 +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