KAFKA-16534 introduced a change to send UpdateVoterRequest every "3 * fetchTimeoutMs" if the voter's configure endpoints are different from the endpoints persisted in the KRaft log. It also introduced a regression where if the voter nodes do not need an update then updateVoterTimer wasn't reset. This resulted in a busy-loop in KafkaRaftClient#poll method resulting in high CPU usage.
This PR modifies the conditions in pollFollowerAsVoter to reset updateVoterTimer appropriately.
Reviewers: José Armando García Sancio <jsancio@apache.org>
Previously, Apache Kafka was uploading release candidate (RC) artifacts
to users' home directories on home.apache.org. However, since this
resource has been decommissioned, we need to follow the standard
approach of putting release candidate artifacts into the appropriate
subversion directory, at https://dist.apache.org/repos/dist/dev/kafka/.
Reviewers: Justine Olshan <jolshan@confluent.io>
When reverting the ZK migration, we must also remove the /migration ZNode in order to allow the migration to be re-attempted in the future.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
When a replica restarts in the follower state it is possible for the set of leader endpoints to not match the latest set of leader endpoints. Voters will discover the latest set of leader endpoints through the BEGIN_QUORUM_EPOCH request. This means that KRaft needs to allow for the replica to transition from Follower to Follower when only the set of leader endpoints has changed.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Alyssa Huang <ahuang@confluent.io>
Kafka Streams system tests were failing with this error:
Failed to parse host name from entry 3001@d for the configuration controller.quorum.voters. Each entry should be in the form `{id}@{host}:{port}`.
The cause is that in kafka.py line 876, we create a delimited string from a list comprehension, but the input is a string itself, so each character gets appended vs. the bootstrap server string of host:port. To fix this, this PR adds split(',') to controller_quorum_bootstrap_servers. Note that this only applies when dynamicRaftQuorum=False
Reviewers: Alyssa Huang <ahuang@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
Fixes a regression introduced by #16669 which inadvertently stopped processing SCRAM arguments from kafka-storage.sh
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Federico Valeri <fedevaleri@gmail.com>
This change fixes a few issues.
KAFKA-17608; KRaft controller crashes when active controller is removed
When a control batch is committed, the quorum controller currently increases the last stable offset but fails to create a snapshot for that offset. This causes an issue if the quorum controller renounces and needs to revert to that offset (which has no snapshot present). Since the control batches are no-ops for the quorum controller, it does not need to update its offsets for control records. We skip handle commit logic for control batches.
KAFKA-17604; Describe quorum output missing added voters endpoints
Describe quorum output will miss endpoints of voters which were added via AddRaftVoter. This is due to a bug in LeaderState's updateVoterAndObserverStates which will pull replica state from observer states map (which does not include endpoints). The fix is to populate endpoints from the lastVoterSet passed into the method.
Reviewers: José Armando García Sancio <jsancio@apache.org>, Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@apache.org>
Several Kafka log configurations in have synonyms. For example, log retention can be configured
either by log.retention.ms, or by log.retention.minutes, or by log.retention.hours. There is also
a faculty in Kafka to dynamically change broker configurations without restarting the broker. These
dynamically set configurations are stored in the metadata log and override what is in the broker
properties file.
Unfortunately, these two features interacted poorly; there was a bug where the dynamic log
configuration update code ignored synonyms. For example, if you set log.retention.minutes and then
reconfigured something unrelated that triggered the LogConfig update path, the retention value that
you had configured was overwritten.
The reason for this was incorrect handling of synonyms. The code tried to treat the Kafka broker
configuration as a bag of key/value entities rather than extracting the correct retention time (or
other setting with overrides) from the KafkaConfig object.
Reviewers: Luke Chen <showuon@gmail.com>, Jun Rao <junrao@gmail.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Christo Lolov <lolovc@amazon.com>, Federico Valeri <fedevaleri@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>, amangandhi94 <>
This test expects that each partition can receive the record, so using a non-null key helps distribute the records more randomly.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This PR tries to improve the error message when broker.id is set to -1 and ZK migration is enabled. It is not
needed to disable the broker.id.generation.enable option. It is sufficient to just not use it (by not setting
the broker.id to -1).
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Luke Chen <showuon@gmail.com>
Change the configurations under config/kraft to use controller.quorum.bootstrap.servers instead of controller.quorum.voters. Add comments explaining how to use the older static quorum configuration where appropriate.
In docs/ops.html, remove the reference to "tentative timelines for ZooKeeper removal" and "Tiered storage is considered as an early access feature" since they are no longer up-to-date. Add KIP-853 information.
In docs/quickstart.html, move the ZK instructions to be after the KRaft instructions. Update the KRaft instructions to use KIP-853.
In docs/security.html, add an explanation of --bootstrap-controller and document controller.quorum.bootstrap.servers instead of controller.quorum.voters.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
This reverts commit 391778b8d7.
Unfortunately that commit re-introduced bug #15127 which prevented the publishing of kafka-clients
artifacts to remote maven. As that bug says:
The issue triggers only with publishMavenJavaPublicationToMavenRepository due to signing.
Generating signed asc files error out for shadowed release artifacts as the module name
(clients) differs from the artifact name (kafka-clients).
The fix is basically to explicitly define artifact of shadowJar to signing and publish plugin.
project.shadow.component(mavenJava) previously outputs the name as client-<version>-all.jar
though the classifier and archivesBaseName are already defined correctly in :clients and
shadowJar construction.
Reviewers: David Arthur <mumrah@gmail.com>
When brokers undergoing ZK migration register with the controller, it should verify that they have
provided a way to contact them via their inter.broker.listener. Otherwise the migration will fail
later on with a more confusing error message.
Reviewers: David Arthur <mumrah@gmail.com>
Failed tasks discovered when removed from the state updater during assignment or revocation are added to the task registry. From there they are retrieved and handled as normal tasks. This leads to a couple of IllegalStateExceptions because it breaks some invariants that ensure that only good tasks are assigned and processed.
This commit solves this bug by distinguish failed from non-failed tasks in the task registry.
Reviewer: Lucas Brutschy <lbrutschy@confluent.io>
There is a race condition between KRaftMigrationDriver running its first poll() and being notified by Raft about a leader change. If onControllerChange is called before RecoverMigrationStateFromZKEvent is run, we will end up getting stuck in the INACTIVE state.
This patch fixes the race by enqueuing a RecoverMigrationStateFromZKEvent from onControllerChange if the driver has not yet initialized. If another RecoverMigrationStateFromZKEvent was already enqueued, the second one to run will just be ignored.
Reviewers: Luke Chen <showuon@gmail.com>
The 3.8 controller assumes the unknown features have min version = 0, but KAFKA-17011 replace the min=0 by min=1 when BrokerRegistrationRequest < 4. Hence, to support upgrading from 3.8.0 to 3.9, this PR changes the implementation of ApiVersionsResponse (<4) and BrokerRegistrationRequest (<4) to skip features with supported minVersion of 0 instead of replacing 0 with 1
Reviewers: Jun Rao <junrao@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
Update the leader before calling handleLeaderChange and use the given epoch in LocalLogManager#prepareAppend. This should hopefully fix several flaky QuorumControllerTest tests.
Reviewers: José Armando García Sancio <jsancio@apache.org>
This patch raises the minimum MetadataVersion for migrations to 3.6-IV1 (metadata transactions). This is only enforced on the controller during bootstrap (when the log is empty). If the log is not empty on controller startup, as in the case of a software upgrade, we allow the migration to continue where it left off.
The broker will log an ERROR message if migrations are enabled and the IBP is not at least 3.6-IV1.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Part of KIP-1033.
Co-authored-by: Dabz <d.gasparina@gmail.com>
Co-authored-by: loicgreffier <loic.greffier@michelin.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>
why df04887ba5 does not fix it?
The fix of df04887ba5 is to NOT collect the log from path `/mnt/kafka/kafka-operational-logs/debug/xxxx.log`if the task is successful. It does not change the log level. see ducktape b2ad7693f2/ducktape/tests/test.py (L181)
why df04887ba5 does not see the error of "sort"
df04887ba5 does NOT show the error since the number of features is only "one" (only metadata.version). Hence, the bug is not triggered as it does not need to "sort". Now, we have two features - metadata.version and krafe.version - so the sort is executed and then we see the "hello bug"
why we should change the kafka.log_level to INFO?
the template of log4j.properties is controlled by `log_level` (https://github.com/apache/kafka/blob/trunk/tests/kafkatest/services/kafka/templates/log4j.properties#L16), and the bug happens in writing debug message (e4ca066680/core/src/main/scala/kafka/server/metadata/BrokerMetadataListener.scala (L274)). Hence, changing the log level to DEBUG can avoid triggering the bug.
Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
7496e62434 fixed an error that caused an exception to be thrown on broker startup when debug logs were on. This made it to every version except 3.2.
The Kraft upgrade tests totally turn off debug logs, but I think we only need to remove them for the broken version.
Note: this bug is also present in 3.1, but there is no logging on startup like in subsequent versions.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <david.jacot@gmail.com>
Add the version check to server side for the specific timestamp:
- the version must be >=8 if timestamp=-4L
- the version must be >=9 if timestamp=-5L
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Because of KIP-902 (Upgrade Zookeeper version to 3.8.2), it is not possible to upgrade from a Kafka version
earlier than 2.4 to a version later than 2.4. Therefore, we should not test these upgrade scenarios
in upgrade_test.py. They do happen to work sometimes, but only in the trivial case where we don't
create topics or make changes during the upgrade (which would reveal the ZK incompatibility).
Instead, we should test only supported scenarios.
Reviewers: Reviewers: José Armando García Sancio <jsancio@gmail.com>
The docker image for Native Apache Kafka was introduced with KIP-974 and was first release with 3.8 AK release.
The docker image for Native Apache Kafka is currently intended for local development and testing purposes.
This PR intends to add a logline indicating the same during docker image startup.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
The COPY_SEGMENT_STARTED state segments are counted when calculating remote retention size. This causes unexpected retention size breached segment deletion. This PR fixes it by
1. only counting COPY_SEGMENT_FINISHED and DELETE_SEGMENT_STARTED state segments when calculating remote log size.
2. During copy Segment, if we encounter errors, we will delete the segment immediately.
3. Tests added.
Co-authored-by: Guillaume Mallet <>
Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>, Guillaume Mallet <>
Previously in KRaft mode, we could request an unclean leader election for a specific topic using
the electLeaders API. This PR adds an additional way to trigger unclean leader election when in
KRaft mode via the static controller configuration and various dynamic configurations.
In order to support all possible configuration methods, we have to do a multi-step configuration
lookup process:
1. check the dynamic topic configuration for the topic.
2. check the dynamic node configuration.
3. check the dynamic cluster configuration.
4. check the controller's static configuration.
Fortunately, we already have the logic to do this multi-step lookup in KafkaConfigSchema.java.
This PR reuses that logic. It also makes setting a configuration schema in
ConfigurationControlManager mandatory. Previously, it was optional for unit tests.
Of course, the dynamic configuration can change over time, or the active controller can change
to a different one with a different configuration. These changes can make unclean leader
elections possible for partitions that they were not previously possible for. In order to address
this, I added a periodic background task which scans leaderless partitions to check if they are
eligible for an unclean leader election.
Finally, this PR adds the UncleanLeaderElectionsPerSec metric.
Co-authored-by: Luke Chen showuon@gmail.com
Reviewers: Igor Soarez <soarez@apple.com>, Luke Chen <showuon@gmail.com>
Implement the remaining ForwardingManager metrics from KIP-938: Add more metrics for measuring KRaft performance:
kafka.server:type=ForwardingManager,name=QueueTimeMs.p99
kafka.server:type=ForwardingManager,name=QueueTimeMs.p999
kafka.server:type=ForwardingManager,name=QueueLength
kafka.server:type=ForwardingManager,name=RemoteTimeMs.p99
kafka.server:type=ForwardingManager,name=RemoteTimeMs.p999
Reviewers: Colin P. McCabe <cmccabe@apache.org>
AccessTokenRetrieverFactory uses the value of sasl.oauthbearer.header.urlencode provided by the user, or null if no value was provided for that configuration. When the HttpAccessTokenRetriever is created the JVM attempts to unbox the value into a boolean, a NullPointerException is thrown.
The fix is to explicitly check the Boolean, and if it's null, use Boolean.FALSE.
Reviewers: bachmanity1 <81428651+bachmanity1@users.noreply.github.com>, Chia-Ping Tsai <chia7712@gmail.com>
The given test took 5 seconds as the logic was waiting completely for 5 seconds for the expiration task to be completed. Adding copySegmentLatch countdown after expiration task is over
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
1) When the local.retention.ms/bytes is set to -2, we didn't replace it with the server-side retention.ms/bytes config, so the -2 local retention won't take effect.
2) When setting retention.ms/bytes to -2, we can notice this log message:
```
Deleting segment LogSegment(baseOffset=10045, size=1037087, lastModifiedTime=1724040653922, largestRecordTimestamp=1724040653835) due to local log retention size -2 breach. Local log size after deletion will be 13435280. (kafka.log.UnifiedLog) [kafka-scheduler-6]
```
This is not helpful for users. We should replace -2 with real retention value when logging.
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Add the version check to client side when building ListOffsetRequest for the specific timestamp:
1) the version must be >=8 if timestamp=-4L (EARLIEST_LOCAL_TIMESTAMP)
2) the version must be >=9 if timestamp=-5L (LATEST_TIERED_TIMESTAMP)
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>