Reviewers: Christo Lolov <lolovc@amazon.com>, Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rdagostino@confluent.io>
As described in ticket KAFKA-15689, this PR fixes the logging of a migration event when the expected migration state is wrong.
Signed-off-by: Paolo Patierno <ppatierno@live.com>
Reviewers: Luke Chen <showuon@gmail.com>
Trivial PR to use the INFO level (instead of DEBUG) for logging the state transition during the ZooKeeper to KRaft migration.
I think it's a useful information to be logged without the need for the user to increase the logging level itself.
Signed-off-by: Paolo Patierno <ppatierno@live.com>
Reviewers: Luke Chen <showuon@gmail.com>, hudeqi <1217150961@qq.com>
This field was missed by the initial KIP-919 PR(s). The result is that migrations can't begin since
the controllers will never become ready. This patch fixes that as well as pulls over some fixes
from the 3.6 branch.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This patch adds reconciliation logic to migrating ZK brokers to deal with pending topic deletions as well as missed StopReplicas.
During the hybrid mode of the ZK migration, the KRaft controller is asynchronously sending UMR and LISR to the ZK brokers to propagate metadata. Since this process is essentially "best effort" it is possible for a broker to miss a StopReplicas. The new logic lets the ZK broker examine its local logs compared with the full set of replicas in a "Full" LISR. Any local logs which are not present in the set of replicas in the request are removed from ReplicaManager and marked as "stray".
To avoid inadvertent data loss with this new behavior, the brokers do not delete the "stray" partitions. They will rename the directories and log warning messages during log recovery. It will be up to the operator to manually delete the stray partitions. We can possibly enhance this in the future to clean up old stray logs.
This patch makes use of the previously unused Type field on LeaderAndIsrRequest. This was added as part of KIP-516 but never implemented. Since its introduction, an implicit 0 was sent in all LISR. The KRaft controller will now send a value of 2 to indicate a full LISR (as specified by the KIP). The presence of this value acts as a trigger for the ZK broker to perform the log reconciliation.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This patch introduces preliminary changes for Eligible Leader Replicas (KIP-966)
* New MetadataVersion 16 (3.7-IV1)
* New record versions for PartitionRecord and PartitionChangeRecord
* New tagged fields on PartitionRecord and PartitionChangeRecord
* New static config "eligible.leader.replicas.enable" to gate the whole feature
Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
The PR includes:
* Added a new class of CleanShutdownFile which helps write and read from a clean shutdown file.
* Updated the BrokerRegistration API.
* Client side handling for the broker epoch.
* Minimum work on the controller side.
Reviewers: Jun Rao <junrao@gmail.com>
Since only the active controller is performing the dual-write to ZK during a migration, it should be the only controller
to report the ZkWriteBehindLag metric.
Currently, if the controller fails over during a migration, the previous active controller will incorrectly report its last
value for ZkWriteBehindLag forever. Instead, it should report zero.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
In KIP-778 we introduced the "unsafe" (lossy) downgrade in case metadata has changes in one of the versions between target and current, as defined in MetadataVersion.
The documentation says it is possible:
"Note that the cluster metadata version cannot be downgraded to a pre-production 3.0.x, 3.1.x, or 3.2.x version once it has been upgraded. However, it is possible to downgrade to production versions such as 3.3-IV0, 3.3-IV1, etc."
The command line tool shows that this doesn't work:
bin/kafka-features.sh --bootstrap-server :9092 downgrade --metadata 3.4 --unsafe
Could not downgrade metadata.version to 8. Invalid metadata.version 8. Unsafe metadata downgrade is not supported in this version.
1 out of 1 operation(s) failed.
In addition to unsafe, also safe metadata downgrades are not supported in practice. For example, when you upgrade to 3.5, you land on 3.5-IV2 as metadata version, which has metadata changes and won't let you to downgrade. This is true for every other release at the moment.
This change fixes the documentation to reflect that, and improves the error messages.
Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
Reviewers: Luke Chen <showuon@gmail.com>, Jakub Scholz <github@scholzj.com>
This patch moves the TopicIdPartition from the metadata module to the server-common module so it can be used by the group-coordinator module as well.
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, David Jacot <djacot@confluent.io>
This is one of the steps required for kafka to compile with Java 21.
For each case, one of the following fixes were applied:
1. Suppress warning if fixing would potentially result in an incompatible change (for public classes)
2. Add final to one or more methods so that the escape is not possible
3. Replace method calls with direct field access.
In addition, we also fix a couple of compiler warnings related to deprecated references in the `core` module.
See the following for more details regarding the new lint warning:
https://www.oracle.com/java/technologies/javase/21-relnote-issues.html#JDK-8015831
Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Chris Egerton <chrise@aiven.io>
Endpoint information provided by KafkaConfig can be incomplete in two ways. One is that endpoints
using ephemeral ports will show up as using port 0. Another is that endpoints binding to 0.0.0.0
will show up with a null or blank hostname. Because we were not accounting for this in controller
registration, it was leading to a null pointer dereference when we tried to register a controller
using an endpoint defined as PLAINTEXT://:9092.
This PR adds a ListenerInfo class which can fix both of the causes of incomplete endpoint
information. It also handles serialization to and from various RPC and record formats.
This allows us to remove a lot of boilerplate code and standardize the handling of listeners
between BrokerServer and ControllerServer.
Reviewers: David Arthur <mumrah@gmail.com>
This patch allows broker heartbeat events to be completed while a metadata transaction is in-flight.
More generally, this patch allows any RUNS_IN_PREMIGRATION event to complete while the controller
is in pre-migration mode even if the migration transaction is in-flight.
We had a problem with broker heartbeats timing out because they could not be completed while a large
ZK migration transaction was in-flight. This resulted in the controller fencing all the ZK brokers which
has many undesirable downstream effects.
Reviewers: Akhilesh Chaganti <akhileshchg@users.noreply.github.com>, Colin Patrick McCabe <cmccabe@apache.org>
Implement KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add
Controller Registration. This KIP adds a new version of DescribeClusterRequest which is supported
by KRaft controllers. It also teaches AdminClient how to use this new DESCRIBE_CLUSTER request to
talk directly with the controller quorum. This is all gated behind a new MetadataVersion,
IBP_3_7_IV0.
In order to share the DESCRIBE_CLUSTER logic between broker and controller, this PR factors it out
into AuthHelper.computeDescribeClusterResponse.
The KIP adds three new errors codes: MISMATCHED_ENDPOINT_TYPE, UNSUPPORTED_ENDPOINT_TYPE, and
UNKNOWN_CONTROLLER_ID. The endpoint type errors can be returned from DescribeClusterRequest
On the controller side, the controllers now try to register themselves with the current active
controller, by sending a CONTROLLER_REGISTRATION request. This, in turn, is converted into a
RegisterControllerRecord by the active controller. ClusterImage, ClusterDelta, and all other
associated classes have been upgraded to propagate the new metadata. In the metadata shell, the
cluster directory now contains both broker and controller subdirectories.
QuorumFeatures previously had a reference to the ApiVersions structure used by the controller's
NetworkClient. Because this PR removes that reference, QuorumFeatures now contains only immutable
data. Specifically, it contains the current node ID, the locally supported features, and the list
of quorum node IDs in the cluster.
Reviewers: David Arthur <mumrah@gmail.com>, Ziming Deng <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
A follow-up to https://github.com/apache/kafka/pull/13804.
This follow-up adds the alternative fix approach mentioned in
the PR above - bumping the session timeout used in the test
with 1 second.
Reproducing the flake-out locally has been much harder than
on the CI runs, as neither Gradle with Java 11 or Java 14 nor
IntelliJ with Java 14 could show it, but IntelliJ with Java 11
could occasionally reproduce the failure the first time
immediately after a rebuild. While I was unable to see the
failure with the bumped session timeout, the testing procedure
definitely didn't provide sufficient reassurance for the
fix as even without it often I'd see hundreds of consecutive
successful test runs when the first run didn't fail.
Reviewers: Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>
When starting up a controller for the first time (i.e., with an empty log), it is possible for
MetadataLoader to publish an empty MetadataImage before the activation records of the controller
have been written. While this is not a bug, it could be confusing. This patch closes that gap by
waiting for at least one controller record to be committed before the MetadataLoader starts publishing
images.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
#14083 added support for delegation tokens in KRaft and attached that support to the existing
MetadataVersion 3.6-IV1. This patch moves that support into a separate MetadataVersion 3.6-IV2.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This PR contains three main changes:
- Support for transactions in MetadataLoader
- Abort in-progress transaction during controller failover
- Utilize transactions for ZK to KRaft migration
A new MetadataBatchLoader class is added to decouple the loading of record batches from the
publishing of metadata in MetadataLoader. Since a transaction can span across multiple batches (or
multiple transactions could exist within one batch), some buffering of metadata updates was needed
before publishing out to the MetadataPublishers. MetadataBatchLoader accumulates changes into a
MetadataDelta, and uses a callback to publish to the publishers when needed.
One small oddity with this approach is that since we can "splitting" batches in some cases, the
number of bytes returned in the LogDeltaManifest has new semantics. The number of bytes included in
a batch is now only included in the last metadata update that is published as a result of a batch.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Viktor Somogyi <viktor.somogyi@cloudera.com>
Implement the QuorumController side of KRaft metadata transactions.
As specified in KIP-868, this PR creates a new metadata version, IBP_3_6_IV1, which contains the
three new records: AbortTransactionRecord, BeginTransactionRecord, EndTransactionRecord.
In order to make offset management unit-testable, this PR moves it out of QuorumController.java and
into OffsetControlManager.java. The general approach here is to track the "last stable offset," which is
calculated by looking at the latest committed offset and the in-progress transaction (if any). When
a transaction is aborted, we revert back to this last stable offset. We also revert back to it when
the controller is transitioning from active to inactive.
In a follow-up PR, we will add support for the transaction records in MetadataLoader. We will also
add support for automatically aborting pending transactions after a controller failover.
Reviewers: David Arthur <mumrah@gmail.com>
On the controller, move publishing acls to the Authorizer into a dedicated MetadataPublisher,
AclPublisher. This publisher listens for notifications from MetadataLoader, and receives only
committed data. This brings the controller side in line with how the broker has always worked. It
also avoids some ugly code related to publishing directly from the QuorumController. Most important
of all, it clears the way to implement metadata transactions without worrying about Authorizer
state (since it will be handled by the MetadataLoader, along with other metadata image state).
In AclsDelta, we can remove isSnapshotDelta. We always know when the MetadataLoader is giving us a
snapshot. Also bring AclsDelta in line with the other delta classes, where completeSnapshot
calculates the diff between the previous image and the next one. We don't use this delta (since we
just apply the image directly to the authorizer) but we should have it, for consistency.
Finally, change MockAclMutator to avoid the need to subclass AclControlManager.
Reviewers: David Arthur <mumrah@gmail.com>
Provide the exact record offset to QuorumController.replay() in all cases. There are several situations
where this is useful, such as logging, implementing metadata transactions, or handling broker
registration records.
In the case where the QC is inactive, and simply replaying records, it is easy to compute the exact
record offset from the batch base offset and the record index.
The active QC case is more difficult. Technically, when we submit records to the Raft layer, it can
choose a batch base offset later than the one we expect, if someone else is also adding records.
While the QC is the only entity submitting data records, control records may be added at any time.
In the current implementation, these are really only used for leadership elections. However, this
could change with the addition of quorum reconfiguration or similar features.
Therefore, this PR allows the QC to tell the Raft layer that a record append should fail if it
would have resulted in a batch base offset other than what was expected. This in turn will trigger a
controller failover. In the future, if automatically added control records become more common, we
may wish to have a more sophisticated system than this simple optimistic concurrency mechanism. But
for now, this will allow us to rely on the offset as correct.
In order that the active QC can learn what offset to start writing at, the PR also adds a new
RaftClient#endOffset function.
At the Raft level, this PR adds a new exception, UnexpectedBaseOffsetException. This gets thrown
when we request a base offset that doesn't match the one the Raft layer would have given us.
Although this exception should cause a failover, it should not be considered a fault. This
complicated the exception handling a bit and motivated splitting more of it out into the new
EventHandlerExceptionInfo class. This will also let us unit test things like slf4j log messages a
bit better.
Reviewers: David Arthur <mumrah@gmail.com>, José Armando García Sancio <jsancio@apache.org>
This patch adds several metrics defined in KIP-866:
* MigratingZkBrokerCount: the number of zk brokers registered with KRaft
* ZkWriteDeltaTimeMs: time spent writing MetadataDelta to ZK
* ZkWriteSnapshotTimeMs: time spent writing MetadataImage to ZK
* Adds value 4 for "ZK" to ZkMigrationState
Also fixes a typo in the metric name introduced in #14009 (ZKWriteBehindLag -> ZkWriteBehindLag)
Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Implement some of the metrics from KIP-938: Add more metrics for
measuring KRaft performance.
Add these metrics to QuorumControllerMetrics:
kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount
kafka.controller:type=KafkaController,name=EventQueueOperationsStartedCount
kafka.controller:type=KafkaController,name=EventQueueOperationsTimedOutCount
kafka.controller:type=KafkaController,name=NewActiveControllersCount
Create LoaderMetrics with these new metrics:
kafka.server:type=MetadataLoader,name=CurrentMetadataVersion
kafka.server:type=MetadataLoader,name=HandleLoadSnapshotCount
Create SnapshotEmitterMetrics with these new metrics:
kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedBytes
kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs
Reviewers: Ron Dagostino <rndgstn@gmail.com>
* Adds an exponential backoff to 1m while the controller is waiting for brokers to show up
* Increases one-time logs to INFO
* Adds a summary of the migration records
* Use RecordRedactor for summary of migration batches (TRACE only)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Standardize controller log4j output for replaying important records. The log message should include
word "replayed" to make it clear that this is a record replay. Log the replay of records for ACLs,
client quotas, and producer IDs, which were previously not logged. Also fix a case where we weren't
logging changes to broker registrations.
AclControlManager, ClientQuotaControlManager, and ProducerIdControlManager didn't previously have a
log4j logger object, so this PR adds one. It also converts them to using Builder objects. This
makes junit tests more readable because we don't need to specify paramaters where the test can use
the default (like LogContexts).
Throw an exception in replay if we get another TopicRecord for a topic which already exists.
Example log messages:
INFO [QuorumController id=3000] Replayed a FeatureLevelRecord setting metadata version to 3.6-IV0
DEBUG [QuorumController id=3000] Replayed a ZkMigrationStateRecord which did not alter the state from NONE.
INFO [QuorumController id=3000] Replayed BrokerRegistrationChangeRecord modifying the registration for broker 0: BrokerRegistrationChangeRecord(brokerId=0, brokerEpoch=3, fenced=-1, inControlledShutdown=0)
INFO [QuorumController id=3000] Replayed ClientQuotaRecord for ClientQuotaEntity(entries={user=testkit}) setting request_percentage to 0.99.
Reviewers: Divij Vaidya <diviv@amazon.com>, Ron Dagostino <rndgstn@gmail.com>, David Arthur <mumrah@gmail.com>
Adds additional testing for the various KRaft *Image classes. For every image that we create we already test that we can get there by applying all the records corresponding to that image written out as a list of records. This patch adds more tests to confirm that we can get to each such final image with intermediate stops at all possible intermediate images along the way.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
Fixes a bug where we don't send UMR and LISR requests in dual-write mode when new partitions are created. Prior to this patch, KRaftMigrationZkWriter was mutating the internal data-structures of TopicDelta which prevented MigrationPropagator from sending UMR and LISR for the changed partitions.
Reviewers: David Arthur <mumrah@gmail.com>
This fixes a regression introduced by the previous KAFKA-15109 commit (d0457f7360 on trunk).
Reviewers: Colin P. McCabe <cmccabe@apache.org>, José Armando García Sancio <jsancio@apache.org>
While in migration mode, the KRaft controller must always bump the leader epoch when shrinking an ISR.
This is required to maintain compatibility with the ZK brokers. Without the epoch bump, the ZK brokers
will ignore the partition state change present in the LeaderAndIsrRequest since it would not contain a new
leader epoch.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This PR expands the scope of ApiVersionManager a bit to include returning the current
MetadataVersion and features that are in effect. This is useful in general because that information
needs to be returned in an ApiVersionsResponse. It also allows us to fix the ApiVersionManager
interface so that all subclasses implement all methods of the interface. Having subclasses that
don't implement some methods is dangerous because they could cause exceptions at runtime in
unexpected scenarios.
On the KRaft controller, we were previously performing a read operation in the QuorumController
thread to get the current metadata version and features. With this PR, we now read a volatile
variable maintained by a separate MetadataVersionContextPublisher object. This will improve
performance and simplify the code. It should not change the guarantees we are providing; in both
the old and new scenarios, we need to be robust against version skew scenarios during updates.
Add a Features class which just has a 3-tuple of metadata version, features, and feature epoch.
Remove MetadataCache.FinalizedFeaturesAndEpoch, since it just duplicates the Features class.
(There are some additional feature-related classes that can be consolidated in in a follow-on PR.)
Create a java class, EndpointReadyFutures, for managing the futures associated with individual
authorizer endpoints. This avoids code duplication between ControllerServer and BrokerServer and
makes this code unit-testable.
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
When received LeaderAndIsr request, we'll notify remoteLogManager about this leadership changed to trigger the following workflow. But LeaderAndIsr won't be sent in KRaft mode, instead, the topicDelta will be received.
This PR fixes this issue by getting leader change and follower change from topicDelta, and triggering rlm.onLeadershipChange to notify remote log manager. Adding tests for remote storage enabled cases.
Reviewers: Satish Duggana <satishd@apache.org>
When the KRaft controller removes a replica from the ISR because of the controlled shutdown there is no need for the leader epoch to be increased by the KRaft controller. This is accurate as long as the topic partition leader doesn't add the removed replica back to the ISR.
This change also fixes a bug when computing the HWM. When computing the HWM, replicas that are not eligible to join the ISR but are caught up should not be included in the computation. Otherwise, the HWM will never increase for replica.lag.time.max.ms because the shutting down replica is not sending FETCH request. Without this additional fix PRODUCE requests would timeout if the request timeout is greater than replica.lag.time.max.ms.
Because of the bug above the KRaft controller needs to check the MV to guarantee that all brokers support this bug fix before skipping the leader epoch bump.
Reviewers: David Mao <47232755+splett2@users.noreply.github.com>, Divij Vaidya <diviv@amazon.com>, David Jacot <djacot@confluent.io>
This creates a builder for PartitionRegistration. The motivation for the builder is that the constructor of PartitionRegistration has four arguments all of type int[] which makes it easy to make a mistake when using it.
Reviewers: José Armando García Sancio <jsancio@apache.org>
In this test broker session timeout is configured aggressively low
(to 1 second) so that fencing can happen without much waiting. Then
in the final portion of the test when brokers should not be fenced
heartbeats are sent roughly 2 times in a session timeout window.
However the first time that's done there's other code between
sending the heartbeat and taking the timestamp, and in local tests
that code can take up to 0.5 seconds (1/2 of the session timeout).
That then can result in all brokers being fenced again which would
fail the test.
This change sends a heartbeat just when a timestamp is taken,
which in local tests results flaky failures from 4 out of 50
to 0 out of 50.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
When the active quorum controller encounters an "unexpected" error, such as a NullPointerException,
it currently resigns its leadership. This PR fixes it so that in addition to doing that, it also
increments the metadata error count metric. This will allow us to better track down these errors.
This PR also fixes a minor bug where performing read operations on a standby controller would
result in an unexpected RuntimeException. The bug happened because the standby controller does not
take in-memory snapshots, and read operations were attempting to read from the epoch of the latest
committed offset. The fix is for the standby controller to simply read the latest value of each
data structure. This is always safe, because standby controllers don't contain uncommitted data.
Also, fix a bug where listPartitionReassignments was reading the latest data, rather than data from
the last committed offset.
Reviewers: dengziming <dengziming1993@gmail.com>, David Arthur <mumrah@gmail.com>
Fixed the metadata change events in the Migration component to check correctly for the diff in
existing topic changes and replicate the metadata to the Zookeeper. Also, made the diff check
exhaustive enough to handle the partial writes in Zookeeper when we're try to replicate changes
using a snapshot in the event of Controller failover.
Add migration client and integration tests to verify the change.
Co-authored-by: Akhilesh Chaganti <akhileshchg@users.noreply.github.com>
This patch adds snapshot reconciliation during ZK to KRaft migration. This reconciliation happens whenever a snapshot is loaded by KRaft, or during a controller failover. Prior to this patch, it was possible to miss metadata updates coming from KRaft when dual-writing to ZK.
Internally this adds a new state SYNC_KRAFT_TO_ZK to the KRaftMigrationDriver state machine. The controller passes through this state after the initial ZK migration and each time a controller becomes active.
Logging during dual-write was enhanced to include a count of write operations happening.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
…tryElection()
A CPU profile in a large cluster showed PartitionChangeBuilder.tryElection() taking significant CPU due to logging. We adjust the logging statements in that method for clean elections from DEBUG level to TRACE to mitigate the impact of this logging under normal operations. Unclean elections are now logged at the INFO level rather than DEBUG.
Reviewers: Jason Gustafson <jason@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
This patch fixes the case where a ClientQuota or SCRAM credential was added in KRaft, but not written back to ZK. This missed write only occurred when handling a KRaft snapshot. If the changed quota was processed in a metadata delta (which is the typical case), it would be written to ZK.
Reviewers: David Arthur <mumrah@gmail.com>
When the active KRaft controller is overloaded, it will not be able to process broker heartbeat
requests. Instead, they will be timed out. When using the default configuration, this will happen
if the time needed to process a broker heartbeat climbs above a second for a sustained period.
This, in turn, could lead to brokers being improperly fenced when they are still alive.
With this PR, timed out heartbeats will still update the lastContactNs and metadataOffset of the
broker in the BrokerHeartbeatManager. While we don't generate any records, this should still be
adequate to prevent spurious fencing. We also log a message at ERROR level so that this condition
will be more obvious.
Other small changes in this PR: fix grammar issue in log4j of BrokerHeartbeatManager. Add JavaDoc
for ClusterControlManager#zkMigrationEnabled field. Add builder for ReplicationControlTestContext
to avoid having tons of constructors. Update ClusterControlManager.DEFAULT_SESSION_TIMEOUT_NS to
match the default in KafkaConfig.
Reviewers: Ismael Juma <ijuma@apache.org>, Ron Dagostino <rdagostino@confluent.io>
This patch fixes several small bugs with configuration dual-write during migration.
* Topic configs are not written back to ZK while handling snapshot.
* New broker/topic configs in KRaft that did not exist in ZK will not be written to ZK.
* The sensitive configs are not encoded while writing them to Zookeeper.
* Handle topic configs in ConfigMigrationClient and KRaftMigrationZkWriter#handleConfigsSnapshot
Added tests to ensure we no longer have the above mentioned issues.
Co-authored-by: Akhilesh Chaganti <akhileshchg@users.noreply.github.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Previously, if a user tried to perform an overly large batch operation on the KRaft controller
(such as creating a million topics), we would create a very large number of records in memory. Our
attempt to write these records to the Raft layer would fail, because there were too many to fit in
an atomic batch. This failure, in turn, would trigger a controller failover.
(Note: I am assuming here that no topic creation policy was in place that would prevent the
creation of a million topics. I am also assuming that the user operation must be done atomically,
which is true for all current user operations, since we have not implemented KIP-868 yet.)
With this PR, we fail immediately when the number of records we have generated exceeds the
threshold that we can apply. This failure does not generate a controller failover. We also now
fail with a PolicyViolationException rather than an UnknownServerException.
In order to implement this in a simple way, this PR adds the BoundedList class, which wraps any
list and adds a maximum length. Attempts to grow the list beyond this length cause an exception to
be thrown.
Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ijuma@apache.org>, Divij Vaidya <diviv@amazon.com>
The local variable processedRecordsSize as just left over from another commit and can be safely removed.
Reviewers: Divij Vaidya <diviv@amazon.com> , José Armando García Sancio <jsancio@apache.org>
Fixed a bug during dual write mode where if a user is updating SCRAM records and has no quotas, the SCRAM records will not be written to ZK. Add tests explicitly for this scenario.
Reviewers: David Arthur <mumrah@gmail.com>
Metadata image classes such as MetadataImage, ClusterImage, FeaturesImage, and so forth contain
numerous sub-images. This PR adds a structured way of traversing those sub-images. This is useful
for the metadata shell, and also for implementing toString functions.
In both cases, the previous solution was suboptimal. The metadata shell was previously implemented
in an ad-hoc way by mutating text-based tree nodes when records were replayed. This was difficult
to keep in sync with changes to the record types (for example, we forgot to do this for SCRAM). It
was also pretty low-level, being done at a level below that of the image classes. For toString, it
was difficult to keep the implementations consistent previously, and also support both redacted and
non-redacted output.
The metadata shell directory was getting crowded since we never had submodules for it. This PR
creates glob/, command/, node/, and state/ directories to keep things better organized.
Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
When loading a snapshot during dual-write mode, we were missing the logic to detect new ACLs that
had been added on the KRaft side. This patch adds support for finding those new ACLs as well as tests
to verify the correct behavior.
Reviewers: David Arthur <mumrah@gmail.com>
Use the MetadataVersion from the MetadataImage passed to MetadataPropagator. The ensures the propagator
sends the right versions of UMR, LISR and StopReplica requests when the migration is in DUAL_WRITE mode.
Reviewers: David Arthur <mumrah@gmail.com>
Rename handleSnapshot to handleLoadSnapshot to make it explicit that it is handling snapshot load,
not generation.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
1. add hint in switch item "BROKER_LOGGER" in ConfigResourceExistenceChecker, otherwise, it will be classified as default break and deleted directly. I don’t know if adding hint is better than deleting directly.
2. delete some unused variables and methods.
3. add the "@test" mark to a method in unit test that is forgotten.
Reviewers: dengziming <dengziming1993@gmail.com>
The KRaft controller return empty finalized features in `ApiVersionResponse`, the brokers are not infected by this, so this problem doesn't have any impact currently, but it's worth fixing it to avoid unexpected problems.
And there is a bunch of of confusing methods in `ApiVersionResponse` which are only used in test code, I moved them to TestUtils to make the code more clear, and force everyone to pass in the correct parameters instead of the default zero parameters, for example, empty supported features and empty finalized features.
Reviewers: Luke Chen <showuon@gmail.com>
This patch adds support for handling metadata snapshots while in dual-write mode. Prior to this change, if the active
controller loaded a snapshot, it would get out of sync with the ZK state.
In order to reconcile the snapshot state with ZK, several methods were added to scan through the metadata in ZK to
compute differences with the MetadataImage. Since this introduced a lot of code, I opted to split out a lot of methods
from ZkMigrationClient into their own client interfaces, such as TopicMigrationClient, ConfigMigrationClient, and
AclMigrationClient. Each of these has some iterator method that lets the caller examine the ZK state in a single pass
and without using too much memory.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Luke Chen <showuon@gmail.com>
When creating the QuorumController, log whether ZK migration is enabled.
When applying a feature level record which sets the metadata version, log the metadata version enum
rather than the numeric feature level.
Improve the logging when we replay snapshots in QuorumController. Log both the beginning and the
end of replay.
When TRACE is enabled, log every record that is replayed in QuorumController. Since some records
may contain sensitive information, create RecordRedactor to assist in logging only what is safe to
put in the log4j file.
Add logging to ControllerPurgatory. Successful completions are logged at DEBUG; failures are logged
at INFO, and additions are logged at TRACE.
Remove SnapshotReason.java, SnapshotReasonTest.java, and
QuorumController#generateSnapshotScheduled. They are deadcode now that snapshot generation moved to
org.apache.kafka.image.publisher.SnapshotGenerator.
Reviewers: David Arthur <mumrah@gmail.com>, José Armando García Sancio <jsancio@apache.org>
Handle migrating SCRAM records in ZK when migrating from ZK to KRaft.
This includes handling writing back SCRAM records to ZK while in dual write mode where metadata updates are written to both the KRaft metadata log and to ZK. This allows for rollback of migration to include SCRAM metadata changes.
Reviewers: David Arthur <mumrah@gmail.com>
1. add ZkMigrationReady in apiVersionsResponse
2. check all nodes if ZkMigrationReady are ready before moving to next migration state
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>
Remove a spurious call to fatalFaultHandler accidentally introduced by KAFKA-14805. We should only
invoke the fatal fault handller if we are unable to generate the activation records. If we are
unable to write the activation records, a controller failover should be sufficient to remedy the
situation.
Co-authored-by: Luke Chen showuon@gmail.com
Reviewers: Luke Chen <showuon@gmail.com>, David Arthur <mumrah@gmail.com>
Don't allow setting negative or zero values for quotas. Don't allow SCRAM mechanism names to be
used as client quota names. SCRAM mechanisms are not client quotas. (The confusion arose because of
internal ZK representation details that treated them both as "client configs.")
Add unit tests for ClientQuotaControlManager.isValidIpEntity and
ClientQuotaControlManager.configKeysForEntityType.
This change doesn't affect metadata record application, only input validation. If there are bad
client quotas that are set currently, this change will not alter the current behavior (of throwing
an exception and ignoring the bad quota).
This patch adds the concept of pre-migration mode to the KRaft controller. While in this mode,
the controller will only allow certain write operations. The purpose of this is to disallow metadata
changes when the controller is waiting for the ZK migration records to be committed.
The following ControllerWriteEvent operations are permitted in pre-migration mode
* completeActivation
* maybeFenceReplicas
* writeNoOpRecord
* processBrokerHeartbeat
* registerBroker (only for migrating ZK brokers)
* unregisterBroker
Raft events and other controller events do not follow the same code path as ControllerWriteEvent,
so they are not affected by this new behavior.
This patch also add a new metric as defined in KIP-868: kafka.controller:type=KafkaController,name=ZkMigrationState
In order to support upgrades from 3.4.0, this patch also redefines the enum value of value 1 to mean
MIGRATION rather than PRE_MIGRATION.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
This patch renames from `ControllerPurgatory` to `DeferredEventQueue` and moves it from the `metadata` module to `server-common` module.
Reviewers: Alexandre Dupriez <alexandre.dupriez@gmail.com>, Ziming Deng <dengziming1993@gmail.com>, José Armando García Sancio <jsancio@apache.org>
Currently, StandardAuthorizer uses a R/W lock for maintaining the consistency of data. For the clusters with very high traffic, we will typically see an increase in latencies whenever a write operation comes. The intent of this PR is to get rid of the R/W lock with the help of immutable or persistent collections. Basically, new object references are used to hold the intermediate state of the write operation. After the completion of the operation, the main reference to the cache is changed to point to the new object. Also, for the read operation, the code is changed such that all accesses to the cache for a single read operation are done to a particular cache object only.
In the PR description, you can find the performance of various libraries at the time of both read and write. Read performance is checked with the existing AuthorizerBenchmark. For write performance, a new AuthorizerUpdateBenchmark has been added which evaluates the performance of the addAcl operation.
Reviewers: Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Divij Vaidya <diviv@amazon.com>
Rework UserScramCredentialRecord to store serverKey and StoredKey rather than saltedPassword. This
is necessary to support migration from ZK, since those are the fields we stored in ZK. Update
latest MetadataVersion to IBP_3_5_IV2 and make SCRAM support conditional on this version. Moved
ScramCredentialData.java from org.apache.kafka.image to org.apache.kafka.metadata, which seems more
appropriate.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
topic counts.
Introduces the use of persistent data structures in the KRaft metadata image to avoid copying the entire TopicsImage upon every change. Performance that was O(<number of topics in the cluster>) is now O(<number of topics changing>), which has dramatic time and GC improvements for the most common topic-related metadata events. We abstract away the chosen underlying persistent collection library via ImmutableMap<> and ImmutableSet<> interfaces and static factory methods.
Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>, Purshotam Chauhan <pchauhan@confluent.io>
Although KAFKA-14808 did not affect KRaft mode, it is important to ensure that we have regression
tests in KRaft mode to prevent a similar bug from appearing there in the future. This PR adds two
tests. First, it adds a test that makes sure we handle what happens when a reassignment completes
and none of the new replicas can be made leader. It's important that we dont keep an old replica as
leader. Second, it adds a test that makes sure we handle new reassignments that don't include a
previous assignment replica that was leader.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
The MetadataLoader must call finishSnapshot after loading a snapshot. This function removes
whatever was in the old snapshot that is not in the new snapshot that was just loaded. While this
is not significant when the old snapshot was the empty snapshot, it is important to do when we are
loading a snapshot on top of an existing non-empty image.
In initializeNewPublishers, the newly installed publishers should be given a MetadataDelta based on
MetadataImage.EMPTY, reflecting the fact that they are seeing everything for the first time.
Reviewers: David Arthur <mumrah@gmail.com>
The SnapshotReader exposes the "last contained log time". This is mainly used during snapshot cleanup. The previous implementation used the append time of the snapshot record. This is not accurate as this is the time when the snapshot was created and not the log append time of the last record included in the snapshot.
The log append time of the last record included in the snapshot is store in the header control record of the snapshot. The header control record is the first record of the snapshot.
To be able to read this record, this change extends the RecordsIterator to decode and expose the control records in the Records type.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
This is a small patch to make it so we only create one FeatureControlManager instance in ReplicationControlManagerTest. Currently we create two, which isn't needed. Its also a bit confusing because the ReplicationControlTestContext objects ends up having a different FeatureControlManager reference that the one its own ReplicationControlManager instance has a reference to.
Reviewers: José Armando García Sancio <jsancio@apache.org>, dengziming <dengziming1993@gmail.com>
The MetadataLoader is not supposed to publish metadata updates until we have loaded up to the high
water mark. Previously, this logic was broken, and we published updates immediately. This PR fixes
that and adds a junit test.
Another issue is that the MetadataLoader previously assumed that we would periodically get
callbacks from the Raft layer even if nothing had happened. We relied on this to install new
publishers in a timely fashion, for example. However, in older MetadataVersions that don't include
NoOpRecord, this is not a safe assumption.
Aside from the above changes, also fix a deadlock in SnapshotGeneratorTest, fix the log prefix for
BrokerLifecycleManager, and remove metadata publishers on brokerserver shutdown (like we do for
controllers).
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>
Currently, we have various bits of reassignment logic spread across different classes. For example, ReplicationControlManager contains logic for when a reassignment is in progress, which is duplication in PartitionChangeBuilder. Another example is PartitionReassignmentRevert which contains logic for how to undo/revert a reassignment. The idea here is to move the logic to PartitionReassignmentReplicas so it's more testable and easier to reason about.
Reviewers: José Armando García Sancio <jsancio@apache.org>
This patch refactors the loadCache method in AclAuthorizer to make it reusable by ZkMigrationClient.
The loaded ACLs are converted to AccessControlEntryRecord. I noticed we still have the defunct
AccessControlRecord, so I've deleted it.
Also included here are the methods to write ACL changes back to ZK while in dual-write mode.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Colin P. McCabe <cmccabe@apache.org>
Separate out KRaft controller metrics into two groups: metrics directly managed by the
QuorumController, and metrics handled by an external publisher. This separation of concerns makes
the code easier to reason about, by clarifying what metrics can be changed where.
The external publisher, ControllerServerMetricsPublisher, handles all metrics which are related to
the content of metadata. For example, metrics about number of topics or number of partitions, etc.
etc. It fits into the MetadataLoader metadata publishing framework as another publisher. Since
ControllerServerMetricsPublisher operates off of a MetadataImage, we don't have to create
(essentially) another copy of the metadata in memory, as ControllerMetricsManager. This reduces
memory consumption. Another benefit of operating off of the MetadataImage is that we don't have to
have special handling for each record type, like we do now in ControllerMetricsManager.
Reviewers: David Arthur <mumrah@gmail.com>
Updates ReplicationControlManager and PartitionReassignmentReplicas to use PartitionAssignment.
Reviewers: José Armando García Sancio <jsancio@apache.org>
Standardize KRaft thread names.
- Always use kebab case. That is, "my-thread-name".
- Thread prefixes are just strings, not Option[String] or Optional<String>.
If you don't want a prefix, use the empty string.
- Thread prefixes end in a dash (except the empty prefix). Then you can
calculate thread names as $prefix + "my-thread-name"
- Broker-only components get "broker-$id-" as a thread name prefix. For example, "broker-1-"
- Controller-only components get "controller-$id-" as a thread name prefix. For example, "controller-1-"
- Shared components get "kafka-$id-" as a thread name prefix. For example, "kafka-0-"
- Always pass a prefix to KafkaEventQueue, so that threads have names like
"broker-0-metadata-loader-event-handler" rather than "event-handler". Prior to this PR, we had
several threads just named "EventHandler" which was not helpful for debugging.
- QuorumController thread name is "quorum-controller-123-event-handler"
- Don't set a thread prefix for replication threads started by ReplicaManager. They run only on the
broker, and already include the broker ID.
Standardize KRaft slf4j log prefixes.
- Names should be of the form "[ComponentName id=$id] ". So for a ControllerServer with ID 123, we
will have "[ControllerServer id=123] "
- For the QuorumController class, use the prefix "[QuorumController id=$id] " rather than
"[Controller <nodeId] ", to make it clearer that this is a KRaft controller.
- In BrokerLifecycleManager, add isZkBroker=true to the log prefix for the migration case.
Standardize KRaft terminology.
- All synonyms of combined mode (colocated, coresident, etc.) should be replaced by "combined"
- All synonyms of isolated mode (remote, non-colocated, distributed, etc.) should be replaced by
"isolated".
This patch fixes many small issues to improve error handling and logging during the ZK migration. A test was added
to simulate a ZK session expiration to ensure the correctness of the migration driver.
With this change, ZK errors thrown during the migration will not hit the fault handler registered with with
KRaftMigrationDriver, but they will be logged.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This patch is the first part of KIP-903. It updates the FetchRequest to include the new tagged ReplicaState field which replaces the now deprecated ReplicaId field. The FetchRequest version is bumped to version 15 and the MetadataVersion to 3.5-IV1.
Reviewers: David Jacot <djacot@confluent.io>
This PR refactors MetadataPublisher's interface a bit. There is now an onControllerChange
callback. This is something that some publishers might want. A good example is ZkMigrationClient.
Instead of two different publish functions (one for snapshots, one for log deltas), we now have a single onMetadataUpdate function. Most publishers didn't want to do anything different in those two cases.
The ones that do want to do something different for snapshots can always check the manifest type.
The close function now has a default empty implementation, since most publishers didn't need to do
anything there.
Move the SCRAM logic out of BrokerMetadataPublisher and run it on the controller as well.
On the broker, simply use dynamicClientQuotaPublisher to handle dynamic client quotas changes.
That is what the controller already does, and the code is exactly the same in both cases.
Fix the logging in FutureUtils.waitWithLogging a bit. Previously, when invoked from BrokerServer
or ControllerServer, it did not include the standard "[Controller 123] " style prefix indicating server
name and ID. This was confusing, especially when debugging junit tests.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, David Arthur <mumrah@gmail.com>
Implement KIP-599 controller mutation quotas for the KRaft controller. These quotas apply to create
topics, create partitions, and delete topic operations. They are specified in terms of number of
partitions.
The approach taken here is to reuse the ControllerMutationQuotaManager that is also used in ZK
mode. The quotas are implemented as Sensor objects and Sensor.checkQuotas enforces the quota,
whereas Sensor.record notes that new partitions have been modified. While ControllerApis handles
fetching the Sensor objects, we must make the final callback to check the quotas from within
QuorumController. The reason is because only QuorumController knows the final number of partitions
that must be modified. (As one example, up-to-date information about the number of partitions that
will be deleted when a topic is deleted is really only available in QuorumController.)
For quota enforcement, the logic is already in place. The KRaft controller is expected to set the
throttle time in the response that is embedded in EnvelopeResponse, but it does not actually apply
the throttle because there is no client connection to throttle. Instead, the broker that forwarded
the request is expected to return the throttle value from the controller and to throttle the client
connection. It also applies its own request quota, so the enforced/returned quota is the maximum of
the two.
This PR also installs a DynamicConfigPublisher in ControllerServer. This allows dynamic
configurations to be published on the controller. Previously, they could be set, but they were not
applied. Note that we still don't have a good way to set node-level configurations for isolatied
controllers. However, this will allow us to set cluster configs (aka default node configs) and have
them take effect on the controllers.
In a similar vein, this PR separates out the dynamic client quota publisher logic used on the
broker into DynamicClientQuotaPublisher. We can now install this on both BrokerServer and
ControllerServer. This makes dynamically configuring quotas (such as controller mutation quotas)
possible.
Also add a ducktape test, controller_mutation_quota_test.py.
Reviewers: David Jacot <djacot@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>
This commit adds support to store the SCRAM credentials in a cluster with KRaft quorum servers and
no ZK cluster backing the metadata. This includes creating ScramControlManager in the controller,
and adding support for SCRAM to MetadataImage and MetadataDelta.
Change UserScramCredentialRecord to contain only a single tuple (name, mechanism, salt, pw, iter)
rather than a mapping between name and a list. This will avoid creating an excessively large record
if a single user has many entries. Because record ID 11 (UserScramCredentialRecord) has not been
used before, this is a compatible change. SCRAM will be supported in 3.5-IV0 and later.
This commit does not include KIP-900 SCRAM bootstrapping support, or updating the credential cache
on the controller (as opposed to broker). We will implement these in follow-on commits.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Some minor improvements to the JavaDoc for ZkMigrationState.
Rename MigrationState to MigrationDriverState to avoid confusion with ZkMigrationState.
Remove ClusterImage#zkBrokers. This costs O(num_brokers) time to calculate, but is only ever used
when in migration state. It should just be calculated in the migration code. (Additionally, the
function ClusterImage.zkBrokers() returns something other than ClusterImage#zkBrokers, which is
confusing.)
Also remove ClusterDelta#liveZkBrokerIdChanges. This is only used in one place, and it's easy to
calculate it there. In general we should avoid providing expensive accessors unless absolutely
necessary. Expensive code should look expensive: if people want to iterate over all brokers, they
can write a loop to do that rather than hiding it inside an accessor.
Added the following checks -
* In StandardAuthorizerData.authorize() to fail if `patternType` other than `LITERAL` is passed.
* In AclControlManager.addAcl() to fail if Resource Name is null or empty.
Also, updated `AclAuthorizerTest` includes a lot of tests covering various scenarios that are missing in `StandardAuthorizerTest`. This PR changes the AclAuthorizerTest to run tests for both `zk` and `kraft` modes -
* Rename AclAuthorizerTest -> AuthorizerTest
* Parameterize relevant tests to run for both modes
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
* assertEquals called on array
* Method is identical to its super method
* Simplifiable assertions
* Unused imports
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Divij Vaidya <diviv@amazon.com>
Only send UMR to ZK brokers if the cluster metadata or topic metadata has changed.
Reviewers: Akhilesh C <akhileshchg@users.noreply.github.com>, Colin P. McCabe <cmccabe@apache.org>
The KIP-405 MetadataVersion changes will be released as part of AK 3.5, but were added as BP_3_4_IV1.
This change fixes them to be IBP_3_5_IV0. There is no incompatibility because this feature has not yet
been released. Also set didMetadataChange to false because KRaft metadata log records did not change.
Reviewers: Satish Duggana <satishd@apache.org>, Christo Lolov <christo_lolov@yahoo.com>, Colin P. McCabe <cmccabe@apache.org>
When in migration-from-ZK mode and sending RPCs to ZK-based brokers, the KRaft controller must send
full UpdateMetadataRequests prior to sending full LeaderAndIsrRequests. If the controller sends the
requests in the other order, and the ZK-based broker does not already know about some of the nodes
referenced in the LeaderAndIsrRequest, it will reject the request.
This PR includes an integration test, and a number of other small fixes for dual-write.
Co-authored-by: Akhilesh C <akhileshchg@users.noreply.github.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
The KRaft client expects the offset of the snapshot id to be an end offset. End offsets are
exclusive. The MetadataProvenance type was createing a snapshot id using the last contained offset
which is inclusive. This change fixes that and renames some of the fields to make this difference
more obvious.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
### JIRA
https://issues.apache.org/jira/browse/KAFKA-14612
### Details
Makes sure we emit `ConfigRecord`s for a topic iff it actually gets created. Currently, we might emit `ConfigRecord`s even if the topic creation fails later in the `createTopics` method.
I created a new method `incrementalAlterConfig` in `ConfigurationControlManager` that is similar to `incrementalAlterConfig` but it just handles one config at a time. This is used in `ReplicationControlManager` for each topic. By handling one topic's config at a time, it's easier to isolate each topic's config records. This enables us to make sure we only write config records for topics that get created.
I refactored `incrementalAlterConfigResource` to return an `ApiError`. This made it easier to implement the new method `incrementalAlterConfig` in `ConfigurationControlManager` because it then doesnt have to search in the `Map` for the result.
### Testing
Enhanced pre-existing test `ReplicationControlManagerTest.testCreateTopicsWithConfigs`. I ran the tests without the changes to `ReplicationControlManager` and made sure each assertion ends up failing. Also ran `./gradlew metadata:test --tests org.apache.kafka.controller.ReplicationControlManagerTest`.
Reviewers: Jason Gustafson <jason@confluent.io>
If KafkaEventQueue gets an InterruptedException while waiting for a condition variable, it
currently exits immediately. Instead, it should complete the remaining events exceptionally and
then execute the cleanup event. This will allow us to finish any necessary cleanup steps.
In order to do this, we require the cleanup event to be provided when the queue is contructed,
rather than when it's being shut down.
Also, handle cases where Event#handleException itself throws an exception.
Remove timed shutdown from the event queue code since nobody was using it, and it adds complexity.
Add server-common/src/test/resources/test/log4j.properties since this gradle module somehow avoided
having a test log4j.properties up to this point.
Reviewers: David Arthur <mumrah@gmail.com>
With the new broker epoch validation logic introduced in #12998, we no longer need the ZK broker epoch to be sent to the KRaft controller. This patch removes that epoch and replaces it with a boolean.
Another small fix is included in this patch for controlled shutdown in migration mode. Previously, if a ZK broker was in migration mode, it would always try to do controlled shutdown via BrokerLifecycleManager. Since there is no ordering dependency between bringing up ZK brokers and the KRaft quorum during migration, a ZK broker could be running in migration mode, but talking to a ZK controller. A small check was added to see if the current controller is ZK or KRaft before decided which controlled shutdown to attempt.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Implements `toString` method for classes `TopicAssignment` and` PartitionAssignment`. Also removes the `final` keyword from the constructor arguments for consistency.
Reviewers: José Armando García Sancio <jsancio@apache.org>
This patch introduces a preliminary state machine that can be used by KRaft
controller to drive online migration from Zk to KRaft.
MigrationState -- Defines the states we can have while migration from Zk to
KRaft.
KRaftMigrationDriver -- Defines the state transitions, and events to handle
actions like controller change, metadata change, broker change and have
interfaces through which it claims Zk controllership, performs zk writes and
sends RPCs to ZkBrokers.
MigrationClient -- Interface that defines the functions used to claim and
relinquish Zk controllership, read to and write from Zk.
Co-authored-by: David Arthur <mumrah@gmail.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
In KRaft, when controller failed to handle events, we'll log error and return back to brokers. But in some cases, we only log error class name, and return error class name back to brokers, which is un-useful for troubleshooting. Ex: When broker registration failed with unsupported version error, it showed:
2022-12-28T17:46:42.876+0800 [DEBUG] [TestEventLogger] [2022-12-28 17:46:42,877] INFO [Controller 3000] registerBroker: failed with UnsupportedVersionException in 2888 us (org.apache.kafka.controller.QuorumController:447)
2022-12-28T17:46:42.877+0800 [DEBUG] [TestEventLogger] [2022-12-28 17:46:42,878] INFO [BrokerLifecycleManager id=0] Unable to register broker 0 because the controller returned error UNSUPPORTED_VERSION (kafka.server.BrokerLifecycleManager:66)
Checking the logs, we still don't know which version it supports.
After this PR, it will show:
2022-12-28T17:54:59.671+0800 [DEBUG] [TestEventLogger] [2022-12-28 17:54:59,671] INFO [Controller 3000] registerBroker: failed with UnsupportedVersionException in 291 us. Reason: Unable to register because the broker does not support version 8 of metadata.version. It wants a version between 4 and 4, inclusive. (org.apache.kafka.controller.QuorumController:447)
2022-12-28T17:54:59.671+0800 [DEBUG] [TestEventLogger] [2022-12-28 17:54:59,672] INFO [BrokerLifecycleManager id=0] Unable to register broker 0 because the controller returned error UNSUPPORTED_VERSION (kafka.server.BrokerLifecycleManager:66)
Reviewers: dengziming <dengziming1993@gmail.com>, Federico Valeri <fvaleri@redhat.com >
Add infrastructure for sending UpdateMetadataRequest and LeaderAndIsr RPCs during the migration
process from ZK to KRaft. The new classes use ControllerChannelManager to send the RPCs. The
information to send comes from MetadataDelta and MetadataImage.
Reviewers: David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Additional notable changes to fix multiple dependency ordering issues:
* Moved `ConfigSynonym` to `server-common`
* Moved synonyms from `LogConfig` to `ServerTopicConfigSynonyms `
* Removed `LogConfigDef` `define` overrides and rely on
`ServerTopicConfigSynonyms` instead.
* Moved `LogConfig.extractLogConfigMap` to `KafkaConfig`
* Consolidated relevant defaults from `KafkaConfig`/`LogConfig` in the latter
* Consolidate relevant config name definitions in `TopicConfig`
* Move `ThrottledReplicaListValidator` to `storage`
Reviewers: Satish Duggana <satishd@apache.org>, Mickael Maison <mickael.maison@gmail.com>
The controller metrics in the controllers has three problems. 1) the active controller exposes uncommitted data in the metrics. 2) the active controller doesn't update the metrics when the uncommitted data gets aborted. 3) the controller doesn't update the metrics when the entire state gets reset.
We fix these issues by only updating the metrics when processing committed metadata records and reset the metrics when the metadata state is reset.
This change adds a new type `ControllerMetricsManager` which processes committed metadata records and updates the metrics accordingly. This change also removes metrics updating responsibilities from the rest of the controller managers.
Reviewers: Ron Dagostino <rdagostino@confluent.io>
This PR implements the follower fetch protocol as mentioned in KIP-405.
Added a new version for ListOffsets protocol to receive local log start offset on the leader replica. This is used by follower replicas to find the local log star offset on the leader.
Added a new version for FetchRequest protocol to receive OffsetMovedToTieredStorageException error. This is part of the enhanced fetch protocol as described in KIP-405.
We introduced a new field locaLogStartOffset to maintain the log start offset in the local logs. Existing logStartOffset will continue to be the log start offset of the effective log that includes the segments in remote storage.
When a follower receives OffsetMovedToTieredStorage, then it tries to build the required state from the leader and remote storage so that it can be ready to move to fetch state.
Introduced RemoteLogManager which is responsible for
initializing RemoteStorageManager and RemoteLogMetadataManager instances.
receives any leader and follower replica events and partition stop events and act on them
also provides APIs to fetch indexes, metadata about remote log segments.
Followup PRs will add more functionality like copying segments to tiered storage, retention checks to clean local and remote log segments. This will change the local log start offset and make sure the follower fetch protocol works fine for several cases.
You can look at the detailed protocol changes in KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
Co-authors: satishd@apache.org, kamal.chandraprakash@gmail.com, yingz@uber.com
Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Cong Ding <cong@ccding.com>, Tirtha Chatterjee <tirtha.p.chatterjee@gmail.com>, Yaodong Yang <yangyaodong88@gmail.com>, Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Jun Rao <junrao@gmail.com>
This PR introduces the new metadata loader and snapshot generator. For the time being, they are
only used by the controller, but a PR for the broker will come soon.
The new metadata loader supports adding and removing publishers dynamically. (In contrast, the old
loader only supported adding a single publisher.) It also passes along more information about each
new image that is published. This information can be found in the LogDeltaManifest and
SnapshotManifest classes.
The new snapshot generator replaces the previous logic for generating snapshots in
QuorumController.java and associated classes. The new generator is intended to be shared between
the broker and the controller, so it is decoupled from both.
There are a few small changes to the old snapshot generator in this PR. Specifically, we move the
batch processing time and batch size metrics out of BrokerMetadataListener.scala and into
BrokerServerMetrics.scala.
Finally, fix a case where we are using 'is' rather than '==' for a numeric comparison in
snapshot_test.py.
Reviewers: David Arthur <mumrah@gmail.com>
Prior to starting a KIP-866 migration, the ZK brokers must register themselves with the active
KRaft controller. The controller waits for all brokers to register in order to verify that all the
brokers can
A) Communicate with the quorum
B) Have the migration config enabled
C) Have the proper IBP set
This patch uses the new isMigratingZkBroker field in BrokerRegistrationRequest and
RegisterBrokerRecord. The type was changed from int8 to bool for BrokerRegistrationRequest (a
mistake from #12860). The ZK brokers use the existing BrokerLifecycleManager class to register and
heartbeat with the controllers.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Let `RaftClient.createSnapshot` take the snapshotId directly instead of the committed offset/epoch (which may not exist).
Reviewers: José Armando García Sancio <jsancio@apache.org>
Introduce MetadataProvenance to encapsulate the three-tuple of (offset, epoch, timestamp) that is
associated with each MetadataImage, as well as each on-disk snapshot. Also introduce a builder
for MetadataDelta.
Remove offset and epoch tracking from MetadataDelta. We do not really need to know this information
until we are creating the final MetadataImage object. Therefore, this bookkeeping should be done by
the metadata loading code, not inside the delta code, like the other bookkeeping. This simplifies a
lot of tests, as well as simplifying RecordTestUtils. It also makes more sense for snapshots, where
the offset and epoch are the same for every record.
Add ImageReWriter, an ImageWriter that applies records to a MetadataDelta. This is useful when you
need to create a MetadataDelta object that holds the contents of a MetadataImage. This will be
used in the new image loader code (coming soon).
Add ImageWriterOptionsTest to test ImageWriterOptions.
Reviewers: David Arthur <mumrah@gmail.com>
This patch adds support for reading and writing ZooKeeper metadata during a KIP-866 migration.
For reading metadata from ZK, methods from KafkaZkClient and ZkData are reused to ensure we are decoding the JSON consistently.
For writing metadata, we use a new multi-op transaction that ensures only a single controller is writing to ZK. This is similar to the existing multi-op transaction that KafkaController uses, but it also includes a check on the new "/migration" ZNode. The transaction consists of three operations:
* CheckOp on /controller_epoch
* SetDataOp on /migration with zkVersion
* CreateOp/SetDataOp/DeleteOp (the actual operation being applied)
In the case of a batch of operations (such as topic creation), only the final MultiOp has a SetDataOp on /migration while the other requests use a CheckOp (similar to /controller_epoch).
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>
This changes the ReplicaPlacer interface to return a class instead of a list of list of integers. There are two reasons for the suggestion. First, as mentioned in the JIRA, it will make the interface, arguably, a bit more readable and understandable by explicitly modeling the idea of topic and partition. Second and more importantly, it makes the interface more extendable in the future. Right now it would be challenging to add more metadata to the response.
Reviewers: José Armando García Sancio <jsancio@apache.org>
Extract jointly owned parts of BrokerServer and ControllerServer into SharedServer. Shut down
SharedServer when the last component using it shuts down. But make sure to stop the raft manager
before closing the ControllerServer's sockets.
This PR also fixes a memory leak where ReplicaManager was not removing some topic metric callbacks
during shutdown. Finally, we now release memory from the BatchMemoryPool in KafkaRaftClient#close.
These changes should reduce memory consumption while running junit tests.
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Implement time based snapshot for the controller. The general strategy for this feature is that the controller will use the record-batch's append time to determine if a snapshot should be generated. If the oldest record that has been committed but is not included in the latest snapshot is older than `metadata.log.max.snapshot.interval.ms`, the controller will trigger a snapshot immediately. This is useful in case the controller was offline for more that `metadata.log.max.snapshot.interval.ms` milliseconds.
If the oldest record that has been committed but is not included in the latest snapshot is NOT older than `metadata.log.max.snapshot.interval.ms`, the controller will schedule a `maybeGenerateSnapshot` deferred task.
It is possible that when the controller wants to generate a new snapshot, either because of time or number of bytes, the controller is currently generating a snapshot. In this case the `SnapshotGeneratorManager` was changed so that it checks and potentially triggers another snapshot when the currently in-progress snapshot finishes.
To better support this feature the following additional changes were made:
1. The configuration `metadata.log.max.snapshot.interval.ms` was added to `KafkaConfig` with a default value of one hour.
2. `RaftClient` was extended to return the latest snapshot id. This snapshot id is used to determine if a given record is included in a snapshot.
3. Improve the `SnapshotReason` type to support the inclusion of values in the message.
Reviewers: Jason Gustafson <jason@confluent.io>, Niket Goel <niket-goel@users.noreply.github.com>
We should avoid using Message.highestSupportedVersion to generate metadata records. Instead, we
need to pick the correct record version based on the current metadata version which is in effect.
In cases where there is only one record version that we know how to generate, we can hard code
that version, but it should just be a simple constant zero.
Reviewers: Jason Gustafson <jason@confluent.io>
This path moves the timeline data structures from metadata module to server-common module as those will be used in the new group coordinator.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Colin Patrick McCabe <cmccabe@apache.org>
This patch adds a unit test for topic recreation with colliding characters (such as `.`). This was broken up until https://github.com/apache/kafka/pull/12790.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>
In https://github.com/apache/kafka/pull/11910 , we added a feature to prevent topics with conflicting metrics names from being created. We added a map to store the normalized topic name to the topic names, but we didn't remove it correctly while deleting topics. This PR fixes this bug and add a test.
Reviewers: Igor Soarez <i@soarez.me>, dengziming <dengziming1993@gmail.com>, Jason Gustafson <jason@confluent.io>
`PartitionRegistration.hashCode` passes raw arrays to `Objects.hash` in the `hashCode` implementation. This doesn't work since the `equals` implementation uses `Arrays.equals`. We should use `Arrays.hashCode` instead.
Reviewers: David Arthur <mumrah@gmail.com>
The iterator `FeatureControlIterator.hasNext()` checks two conditions: 1) whether we have already written the metadata version, and 2) whether the underlying iterator has additional records. However, in `next()`, we also check that the metadata version is at least high enough to include it in the log. When this fails, then we can see an unexpected `NoSuchElementException` if the underlying iterator is empty.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
Setting the `committedBytesSinceLastSnapshot` to 0 when resigning can cause the controller to not generate a snapshot after `snapshotMaxNewRecordBytes` committed bytes have been replayed.
This change fixes that by simply not resetting the counter during resignation. This is correct because the counter tracks the number of committed bytes replayed but not included in the latest snapshot. In other words, reverting the last committed state does not invalidate this value.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
The `controlledShutDownOffset` is defined as the "offset at which the broker should complete its controlled shutdown, or -1 if the broker is not performing a controlled shutdown". The controller sets this offset to a non-negative integer on receiving a heartbeat from a broker that's in controlled shutdown state. Currently, this offset is being updated and bumped every single time a broker in controlled shutdown mode send a heartbeat, delaying when controlled shutdown can actually complete for the broker. We should only update the offset when it was previously set to -1 to allow controlled shutdown to complete.
Reviewers: Luke Chen <showuon@gmail.com>, Jason Gustafson <jason@confluent.io>
This PR adds a new ImageWriter interface which replaces the generic Consumer interface which
accepted lists of records. It is better to do batching in the ImageWriter than to try to deal with
that complexity in the MetadataImage#write functions, especially since batching is not semantically
meaningful in KRaft snapshots. The new ImageWriter interface also supports freeze and close, which
more closely matches the semantics of the underlying Raft classes.
The PR also adds an ImageWriterOptions class which we can use to pass parameters to control how the
new image is written. Right now, the parameters that we are interested in are the target metadata
version (which may be more or less than the original image's version) and a handler function which
is invoked whenever metadata is lost due to the target version.
Convert over the MetadataImage#write function (and associated functions) to use the new ImageWriter
and ImageWriterOptions. In particular, we now have a way to handle metadata losses by invoking
ImageWriterOptions#handleLoss. This allows us to handle writing an image at a lower version, for
the first time. This support is still not enabled externally by this PR, though. That will come in
a future PR.
Get rid of the use of SOME_RECORD_TYPE.highestSupportedVersion() in several places. In general, we
do not want to "silently" change the version of a record that we output, just because a new version
was added. We should be explicit about what record version numbers we are outputting.
Implement ProducerIdsDelta#toString, to make debug logs look better.
Move MockRandom to the server-common package so that other internal broker packages can use it.
Reviewers: José Armando García Sancio <jsancio@apache.org>
Make all faults in metadata processing on standby controllers be fatal. This is the same behavior-wise as the active controller. This prevents a standby controller from eventually becoming active with incomplete state.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
Add builders for LocalLogManagerTestEnv and QuorumControllerTestEnv, since the constructor
overloads were starting to get unwieldy.
Make indentation more consistent in QuorumControllerTest. Take advantage of the fact that you can
initialize multiple resources in a Java try-with-resources block to avoid excessive indentation in a few
cases.
Reviewers: José Armando García Sancio <jsancio@apache.org>
In https://github.com/apache/kafka/pull/12695, we discovered a gap in our testing of `StandardAuthorizer`. We addressed the specific case that was failing, but I think we need to establish a better methodology for testing which incorporates randomized inputs. This patch is a start in that direction. We implement a few basic property tests using jqwik which focus on prefix searching. It catches the case from https://github.com/apache/kafka/pull/12695 prior to the fix. In the future, we can extend this to cover additional operation types, principal matching, etc.
Reviewers: David Arthur <mumrah@gmail.com>
Previously, BrokerRegistration#toString sould throw an exception, terminating metadata replay,
because the sorted() method is used on an entry set rather than a key set.
Reviewers: David Arthur <mumrah@gmail.com>
Fixes an issue with StandardAuthorizer#authorize that allowed inconsistent results. The underlying
concurrent data structure (ConcurrentSkipListMap) had weak consistency guarantees. This meant
that a concurrent update to the authorizer data could result in the authorize function processing
ACL updates out of order.
This patch replaces the concurrent data structures with regular non-thread safe equivalents and uses
a read/write lock for thread safety and strong consistency.
Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>, Colin P. McCabe <cmccabe@apache.org>, Luke Chen <showuon@gmail.com>
When a snapshot is taken it is due to either of the following reasons -
Max bytes were applied
Metadata version was changed
Once the snapshot process is started, it will log the reason that initiated the process.
Updated existing tests to include code changes required to log the reason. I was not able to check the logs when running tests - could someone guide me on how to enable logs when running a specific test case.
Reviewers: dengziming <dengziming1993@gmail.com>, José Armando García Sancio <jsancio@apache.org>
Originally, the QuorumController did not try to limit the number of records in a batch that it sent
to the Raft layer. This caused two problems. Firstly, we were not correctly handling the exception
that was thrown by the Raft layer when a batch of records was too large to apply atomically. This
happened because the Raft layer threw an exception which was a subclass of ApiException. Secondly,
by letting the Raft layer split non-atomic batches, we were not able to create snapshots at each of
the splits. This led to O(N) behavior during controller failovers.
This PR fixes both of these issues by limiting the number of records in a batch. Atomic batches
that are too large will fail with a RuntimeException which will cause the active controller to
become inactive and revert to the last committed state. Non-atomic batches will be split into
multiple batches with a fixed number of records in each.
Reviewers: Luke Chen <showuon@gmail.com>, José Armando García Sancio <jsancio@gmail.com>
#12374 adjusted the invocation of the alter configs policy check in KRaft to match the behavior in ZooKeeper, which is to only provide the configs that were explicitly sent in the request. While the code was correct for the incremental alter configs case, the code actually included the implicit deletions for the legacy/non-incremental alter configs case, and those implicit deletions are not included in the ZooKeeper-based invocation. This patch adds a test to check for this and adjusts ConfigurationControlManager code so that the test passes -- the adjusted test is confirmed to fail locally otherwise. We also add a log statement to emit any unexpected stack traces in the alter config code path.
Reviewers: José Armando García Sancio <jsancio@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
The boostrap.checkpoint files should include a control record batch for
the SnapshotHeaderRecord at the start of the file. It should also
include a control record batch for the SnapshotFooterRecord at the end
of the file.
The snapshot header record is important because it versions the rest of
the bootstrap file.
Reviewers: David Arthur <mumrah@gmail.com>
The main changes here are ensuring that we always have a metadata.version record in the log, making
˘sure that the bootstrap file can be used for records other than the metadata.version record (for
example, we will want to put SCRAM initialization records there), and fixing some bugs.
If no feature level record is in the log and the IBP is less than 3.3IV0, then we assume the minimum KRaft
version for all records in the log.
Fix some issues related to initializing new clusters. If there are no records in the log at all,
then insert the bootstrap records in a single batch. If there are records, but no metadata version,
process the existing records as though they were metadata.version 3.3IV0 and then append a metadata
version record setting version 3.3IV0. Previously, we were not clearly distinguishing between the
case where the metadata log was empty, and the case where we just needed to add a metadata.version
record.
Refactor BootstrapMetadata into an immutable class which contains a 3-tuple of metadata version,
record list, and source. The source field is used to log where the bootstrap metadata was obtained
from. This could be a bootstrap file, the static configuration, or just the software defaults.
Move the logic for reading and writing bootstrap files into BootstrapDirectory.java.
Add LogReplayTracker, which tracks whether the log is empty.
Fix a bug in FeatureControlManager where it was possible to use a "downgrade" operation to
transition to a newer version. Do not store whether we have seen a metadata version or not in
FeatureControlManager, since that is now handled by LogReplayTracker.
Introduce BatchFileReader, which is a simple way of reading a file containing batches of snapshots
that does not require spawning a thread. Rename SnapshotFileWriter to BatchFileWriter to be
consistent, and to reflect the fact that bootstrap files aren't snapshots.
QuorumController#processBrokerHeartbeat: add an explanatory comment.
Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
AccessControlEntryRecord and RemoveAccessControlEntryRecord are added in KIP-801, FeatureLevelRecord was added in KIP-778, and BrokerRegistrationChangeRecord was added in KIP-841, and NoOpRecord was added in KIP-835, I added these 5 record types in MetadataShell.
Reviewers: Luke Chen <showuon@gmail.com>
Catch InvocationTargetException explicitly and propagate underlying cause
Reviewers: Ismael Juma <mlists@juma.me.uk>, Matthew de Detrich <mdedetrich@gmail.com>, Kvicii, Luke Chen <showuon@gmail.com>
Fix a bug in ReplicationControlManager where we got a NullPointerException when removing a topic
with no offline replicas, and there were other topics that did have offline replicas.
Fix an issue in MetadataDelta#replay where we were replaying RemoveTopicRecord twice.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>
Similar to https://github.com/apache/kafka/pull/12506. For the Kraft controller, we should return NOT_CONTROLLER if the leader/partition epoch in the request is ahead of the controller.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>
The reason for KAFKA-13959 is a little complex, the two keys to this problem are:
KafkaRaftClient.MAX_FETCH_WAIT_MS==MetadataMaxIdleIntervalMs == 500ms. We rely on fetchPurgatory to complete a FetchRequest, in details, if FetchRequest.fetchOffset >= log.endOffset, we will wait for 500ms to send a FetchResponse. The follower needs to send one more FetchRequest to get the HW.
Here are the event sequences:
1. When starting the leader(active controller) LEO=m+1(m is the offset of the last record), leader HW=m(because we need more than half of the voters to reach m+1)
2. Follower (standby controller) and observer (broker) send FetchRequest(fetchOffset=m)
2.1. leader receives FetchRequest, set leader HW=m and waits 500ms before send FetchResponse
2.2. leader send FetchResponse(HW=m)
3.3 broker receive FetchResponse(HW=m), set metadataOffset=m.
3. Leader append NoOpRecord, LEO=m+2. leader HW=m
4. Looping 1-4
If we change MAX_FETCH_WAIT_MS=200 (less than half of MetadataMaxIdleIntervalMs), this problem can be solved temporarily.
We plan to improve this problem in 2 ways, firstly, in this PR, we change the controller to unfence a broker when the broker's high-watermark has reached the broker registration record for that broker. Secondly, we will propagate the HWM to the replicas as quickly as possible in KAFKA-14145.
Reviewers: Luke Chen <showuon@gmail.com>, José Armando García Sancio <jsancio@users.noreply.github.com>
This PR adds in 3 metrics as described in KIP-859:
kafka.server:type=broker-metadata-metrics,name=metadata-load-error-count
kafka.server:type=broker-metadata-metrics,name=metadata-apply-error-count
kafka.controller:type=KafkaController,name=MetadataErrorCount
These metrics are incremented by fault handlers when the appropriate fault happens. Broker-side
load errors happen in BrokerMetadataListener. Broker-side apply errors happen in the
BrokerMetadataPublisher. The metric on the controller is incremented when the standby controller
(not active) encounters a metadata error.
In BrokerMetadataPublisher, try to limit the damage caused by an exception by introducing more
catch blocks. The only fatal failures here are those that happen during initialization, when we
initialize the manager objects (these would also be fatal in ZK mode).
In BrokerMetadataListener, try to improve the logging of faults, especially ones that happen when
replaying a snapshot. Try to limit the damage caused by an exception.
Replace MetadataFaultHandler with LoggingFaultHandler, which is more flexible and takes a Runnable
argument. Add LoggingFaultHandlerTest.
Make QuorumControllerMetricsTest stricter. Fix a bug where we weren't cleaning up some metrics from
the yammer registry on close in QuorumControllerMetrics.
Co-author: Colin P. McCabe <cmccabe@apache.org>
This commit adds a check to ensure the RecordBatch CRC is valid when
iterating over a Batch of Records using the RecordsIterator. The
RecordsIterator is used by both Snapshot reads and Log Records reads in
Kraft. The check can be turned off by a class parameter and is on by default.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>
Before trying to commit a batch of records to the __cluster_metadata log, the active controller
should try to apply them to its current in-memory state. If this application process fails, the
active controller process should exit, allowing another node to take leadership. This will prevent
most bad metadata records from ending up in the log and help to surface errors during testing.
Similarly, if the active controller attempts to renounce leadership, and the renunciation process
itself fails, the process should exit. This will help avoid bugs where the active controller
continues in an undefined state.
In contrast, standby controllers that experience metadata application errors should continue on, in
order to avoid a scenario where a bad record brings down the whole controller cluster. The
intended effect of these changes is to make it harder to commit a bad record to the metadata log,
but to continue to ride out the bad record as well as possible if such a record does get committed.
This PR introduces the FaultHandler interface to implement these concepts. In junit tests, we use a
FaultHandler implementation which does not exit the process. This allows us to avoid terminating
the gradle test runner, which would be very disruptive. It also allows us to ensure that the test
surfaces these exceptions, which we previously were not doing (the mock fault handler stores the
exception).
In addition to the above, this PR fixes a bug where RaftClient#resign was not being called from the
renounce() function. This bug could have resulted in the raft layer not being informed of an active
controller resigning.
Reviewers: David Arthur <mumrah@gmail.com>
KRaft should validate that manual assignments given to createTopics are contiguous. In other words,
they must start with partition 0, and progress through 1, 2, 3, etc. ZK mode does this, but KRaft
mode previously did not. Also fix a null pointer exception when the placement for partition 0
was not specified.
Convert over AddPartitionsTest to use KRaft. This PR converts all of the test except for some of
the placement logic tests, which will need to be redone for KRaft mode in a future change.
Fix null pointer exception in KRaftMetadataCache#getPartitionInfo. Specifically, we should not
assume that the partition will be found in the hash map. This is another case where we had
"Some(x)" but it should be "Option(x)."
Fix a potential null pointer exception in BrokerServer#state.
Reviewers: dengziming <dengziming1993@gmail.com>, Jason Gustafson <jason@confluent.io>
The code used BrokerRegistrationFencingChange.FENCE when unfencing a broker and used BrokerRegistrationFencingChange.UNFENCE when fencing a broker, this is confusing. This commit flips the values of the two enums and changes their usage at all of the call sites.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>
Only pass configs from the request to the AlterConfigPolicy. This changes the KRaft usage of the AlterConfigPolicy to match the usage in ZK mode.
Reviewers: Jason Gustafson <jason@confluent.io>
Fixes two issues in the implementation of `LocalLogManager`:
- As per the interface contract for `RaftClient.scheduleAtomicAppend()`, it should throw a `NotLeaderException` exception when the provided current leader epoch does not match the current epoch. However, the current `LocalLogManager`'s implementation of the API returns a LONG_MAX instead of throwing an exception. This change fixes the behaviour and makes it consistent with the interface contract.
- As per the interface contract for `RaftClient.resign(epoch)`if the parameter epoch does not match the current epoch, this call will be ignored. But in the current `LocalLogManager` implementation the leader epoch might change when the thread is waiting to acquire a lock on `shared.tryAppend()` (note that tryAppend() is a synchronized method). In such a case, if a NotALeaderException is thrown (as per code change in above), then resign should be ignored.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Tom Bentley <tbentley@redhat.com>, Jason Gustafson <jason@confluent.io>
There are a bunch of tests which do not clean up after themselves. This leads to
accumulation of files in the tmp directory of the system on which the tests are
running.
This code change fixes some of the main culprit tests which leak the files in the
temporary directory.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Kvicii <kvicii.yu@gmail.com>
This path implements [KIP-841](https://cwiki.apache.org/confluence/display/KAFKA/KIP-841%3A+Fenced+replicas+should+not+be+allowed+to+join+the+ISR+in+KRaft). Specifically, it implements the following:
* It introduces INELIGIBLE_REPLICA and NEW_LEADER_ELECTED error codes.
* The KRaft controller validates the new ISR provided in the AlterPartition request and rejects the call if any replica in the new ISR is not eligible to join the the ISR - e.g. when fenced or shutting down. The leader reverts to the last committed ISR when its request is rejected due to this.
* The partition leader also verifies that a replica is eligible before trying to add it back to the ISR. If it is not eligible, the ISR expansion is not triggered at all.
* Updates the AlterPartition API to use topic ids. Updates the AlterPartition manger to handle topic names/ids. Updates the ZK controller and the KRaft controller to handle topic names/ids depending on the version of the request used.
Reviewers: Artem Livshits <84364232+artemlivshits@users.noreply.github.com>, José Armando García Sancio <jsancio@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
* Set the minimum supported MetadataVersion to 3.0-IV1
* Remove MetadataVersion.UNINITIALIZED
* Relocate RPC version mapping for fetch protocols into MetadataVersion
* Replace static IBP calls with dynamic calls to MetadataCache
A side effect of removing the UNINITIALIZED metadata version is that the FeatureControlManager and FeatureImage will initialize themselves with the minimum KRaft version (3.0-IV1).
The rationale for setting the minimum version to 3.0-IV1 is so that we can avoid any cases of KRaft mode running with an old log message format (KIP-724 was introduced in 3.0-IV1). As a side-effect of increasing this minimum version, the feature level values decreased by one.
Reviewers: Jason Gustafson <jason@confluent.io>, Jun Rao <junrao@gmail.com>
As part of KIP-835, LastCommittedRecordOffset was added to the
KafkaController metric type. Make sure to test that metric.
Reviewers: Jason Gustafson <jason@confluent.io>
This PR implements the first part of KIP-841. Specifically, it implements the following:
1. Adds a new metadata version.
2. Adds the InControlledShutdown field to the BrokerRegistrationRecord and BrokerRegistrationChangeRecord and bump their versions. The newest versions are only used if the new metadata version is enabled.
3. Writes a BrokerRegistrationChangeRecord with InControlledShutdown set when a broker requests a controlled shutdown.
4. Ensures that fenced and in controlled shutdown replicas are not picked as leaders nor included in the ISR.
5. Adds or extends unit tests.
Reviewes: José Armando García Sancio <jsancio@users.noreply.github.com>, dengziming <dengziming1993@gmail.com>, David Arthur <mumrah@gmail.com>
Implement BrokerRegistrationChangeRecord as specified in KIP-746. This is a more flexible record than the
single-purpose Fence / Unfence records.
Reviewers: José Armando García Sancio <jsancio@gmail.com>, dengziming <dengziming1993@gmail.com>
This PR fixes a bug where FeatureControlManager#replay(FeatureLevelRecord) was throwing an
exception if not all controllers in the quorum supported the feature being applied. While we do
want to validate this, it needs to be validated earlier, before the record is committed to the log.
Once the record has been committed to the log it should always be applied if the current controller
supports it.
Fix another bug where removing a feature was not supported once it had been configured. Note that
because we reserve feature level 0 for "feature not enabled", we don't need to use
Optional<VersionRange>; we can just return a range of 0-0 when the feature is not supported.
Allow the metadata version to be downgraded when UpgradeType.UNSAFE_DOWNGRADE has been set.
Previously we were unconditionally denying this even when this was set.
Add a builder for FeatureControlManager, so that we can easily add new parameters to the
constructor in the future. This will also be useful for creating FeatureControlManagers that are
initialized to a specific MetadataVersion.
Get rid of RemoveFeatureLevelRecord, since it's easier to just issue a FeatureLevelRecord with
the level set to 0.
Set metadata.max.idle.interval.ms to 0 in RaftClusterSnapshotTest for more predictability.
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>
Implement NoOpRecord as described in KIP-835. This is controlled by the new
metadata.max.idle.interval.ms configuration.
The KRaft controller schedules an event to write NoOpRecord to the metadata log if the metadata
version supports this feature. This event is scheduled at the interval defined in
metadata.max.idle.interval.ms. Brokers and controllers were improved to ignore the NoOpRecord when
replaying the metadata log.
This PR also addsffour new metrics to the KafkaController metric group, as described KIP-835.
Finally, there are some small fixes to leader recovery. This PR fixes a bug where metadata version
3.3-IV1 was not marked as changing the metadata. It also changes the ReplicaControlManager to
accept a metadata version supplier to determine if the leader recovery state is supported.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This updates StandardAuthorizer and StandardAuthorizerData to use parameterized logging per the SLF4J recommendation (see https://www.slf4j.org/faq.html). This also removes a couple if statements that explicitly check if trace is enabled, but the logger should handle not publishing the message and not constructing the String if trace is not enabled.
Reviewers: Jason Gustafson <jason@confluent.io>
AclsDelta stores the pending deletion in the changes Map. This could override a creation that might have just happened. This is an issue because in BrokerMetadataPublisher this results in us making a removeAcl call which finally results in StandardAuthorizerData trying to remove an ACL that doesn't exist which throws an exception. If the ACCESS_CONTROL_ENTRY_RECORD event never got processed by BrokerMetadataPublisher then the ACL wont be in the Map yet.
The fix here is to remove the entry from the changes Map if the ACL doesnt exist in the image yet.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>
This patch ensures consistent handling of null-valued topic configs between the zk and kraft controller. Prior to this patch, we returned INVALID_REQUEST in zk mode and it was not an error in kraft. After this patch, we return INVALID_CONFIG consistently for this case.
Reviewers: Jason Gustafson <jason@confluent.io>
This patch builds on #12072 and adds controller support for metadata.version. The kafka-storage tool now allows a
user to specify a specific metadata.version to bootstrap into the cluster, otherwise the latest version is used.
Upon the first leader election of the KRaft quroum, this initial metadata.version is written into the metadata log. When
writing snapshots, a FeatureLevelRecord for metadata.version will be written out ahead of other records so we can
decode things at the correct version level.
This also includes additional validation in the controller when setting feature levels. It will now check that a given
metadata.version is supportable by the quroum, not just the brokers.
Reviewers: José Armando García Sancio <jsancio@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>, Alyssa Huang <ahuang@confluent.io>
Since the StandardAuthorizer relies on the metadata log to store its ACLs, we need to be sure that
we have the latest metadata before allowing the authorizer to be used. However, if the authorizer
is not usable for controllers in the cluster, the latest metadata cannot be fetched, because
inter-node communication cannot occur. In the initial commit which introduced StandardAuthorizer,
we punted on the loading issue by allowing the authorizer to be used immediately. This commit fixes
that by implementing early.start.listeners as specified in KIP-801. This will allow in superusers
immediately, but throw the new AuthorizerNotReadyException if non-superusers try to use the
authorizer before StandardAuthorizer#completeInitialLoad is called.
For the broker, we call StandardAuthorizer#completeInitialLoad immediately after metadata catch-up
is complete, right before unfencing. For the controller, we call
StandardAuthorizer#completeInitialLoad when the node has caught up to the high water mark of the
cluster metadata partition.
This PR refactors the SocketServer so that it creates the configured acceptors and processors in
its constructor, rather than requiring a call to SocketServer#startup A new function,
SocketServer#enableRequestProcessing, then starts the threads and begins listening on the
configured ports. enableRequestProcessing uses an async model: we will start the acceptor and
processors associated with an endpoint as soon as that endpoint's authorizer future is completed.
Also fix a bug where the controller and listener were sharing an Authorizer when in co-located
mode, which was not intended.
Reviewers: Jason Gustafson <jason@confluent.io>
Fix a bug where multiple remove records could be generated for a single ACL. Previously, this happened
if the user submitted multiple filters to deleteAcls, and more than one matched.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
We can append/subtract multiple config values in kraft mode using the `IncrementalAlterConfig` RPC. For example: append/subtract topic config "cleanup.policy" with value="delete,compact" will end up treating "delete,compact" as a value not 2 values. This patch fixes the problem. Additionally, it update the zk logic to correctly handle duplicate additions.
Reviewers: Akhilesh Chaganti <akhileshchg@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
The KRaft implementation of the `CreatePartitions` ignores the `validateOnly` flag in the
request and creates the partitions if the validations are successful. Fixed the behavior
not to create partitions upon validation if the `validateOnly` flag is true.
Reviewers: Divij Vaidya <divijvaidya13@gmail.com>, dengziming <dengziming1993@gmail.com>, Jason Gustafson <jason@confluent.io>
This patch does some initial cleanups in the context of KAFKA-13790. Mainly, it renames `ZkVersion` field to `PartitionEpoch` in the `LeaderAndIsrRequest`, the `LeaderAndIsr` and the `Partition`.
Reviewers: Jason Gustafson <jason@confluent.io>, dengziming <dengziming1993@gmail.com>
This PR fixes a case where we were unable to place on fenced brokers In KRaft mode. Specifically,
if we had a broker registration in the metadata log, but no associated heartbeat, previously the
HeartbeatManager would not track the fenced broker. This PR fixes this by adding this logic to the
metadata log replay path in ClusterControlManager.
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>
When LeaderRecoveryState was added to the PartitionChangeRecord, the
check for being a noop was not updated. This commit fixes that and
improves the associated test to avoid this oversight in the future.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
Ensure that we can set log.flush.interval.ms at the broker or cluster level via
IncrementalAlterConfigs. This was broken by KAFKA-13749, which added log.flush.interval.ms as the
second synonym rather than the first. Add a regression test to DynamicConfigChangeTest.
Create ControllerRequestContext and pass it to every controller API. This gives us a uniform way to
pass through information like the deadline (if there is one) and the Kafka principal which is
making the request (in the future we will want to log this information).
In ControllerApis, enforce a timeout for broker heartbeat requests which is equal to the heartbeat
request interval, to avoid heartbeats piling up on the controller queue. This should have been done
previously, but we overlooked it.
Add a builder for ClusterControlManager and ReplicationControlManager to avoid the need to deal
with a lot of churn (especially in test code) whenever a new constructor parameter gets added for
one of these.
In ControllerConfigurationValidator, create a separate function for when we just want to validate
that a ConfigResource is a valid target for DescribeConfigs. Previously we had been re-using the
validation code for IncrementalAlterConfigs, but this was messy.
Split out the replica placement code into a separate package and reorganize it a bit.
Reviewers: David Arthur <mumrah@gmail.com
This PR includes the changes to feature flags that were outlined in KIP-778. Specifically, it
changes UpdateFeatures and FeatureLevelRecord to remove the maximum version level. It also adds
dry-run to the RPC so the controller can actually attempt the upgrade (rather than the client). It
introduces an upgrade type enum, which supersedes the allowDowngrade boolean. Because
FeatureLevelRecord was unused previously, we do not need to introduce a new version.
The kafka-features.sh tool was overhauled in KIP-778 and now includes the describe, upgrade,
downgrade, and disable sub-commands. Refer to
[KIP-778](https://cwiki.apache.org/confluence/display/KAFKA/KIP-778%3A+KRaft+Upgrades) for more
details on the new command structure.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>
In ZK mode, the topic "foo_bar" will conflict with "foo.bar" because of limitations in metric
names. We should implement this in KRaft mode. This PR also changes TopicCommandIntegrationTest to
support KRaft mode.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This patch adds audit support through the kafka.authorizer.logger logger to StandardAuthorizer. It
follows the same conventions as AclAuthorizer with a similarly formatted log message. When
logIfAllowed is set in the Action, then the log message is at DEBUG level; otherwise, we log at
trace. When logIfDenied is set, then the log message is at INFO level; otherwise, we again log at
TRACE.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Previously, when in KRaft mode, CreateTopics did not return the active configurations for the
topic(s) it had just created. This PR addresses that gap. We will now return these topic
configuration(s) when the user has DESCRIBE_CONFIGS permission. (In the case where the user does
not have this permission, we will omit the configurations and set TopicErrorCode. We will also omit
the number of partitions and replication factor data as well.)
For historical reasons, we use different names to refer to each topic configuration when it is set
in the broker context, as opposed to the topic context. For example, the topic configuration
"segment.ms" corresponds to the broker configuration "log.roll.ms". Additionally, some broker
configurations have synonyms. For example, the broker configuration "log.roll.hours" can be used to
set the log roll time instead of "log.roll.ms". In order to track all of this, this PR adds a
table in LogConfig.scala which maps each topic configuration to an ordered list of ConfigSynonym
classes. (This table is then passed to KafkaConfigSchema as a constructor argument.)
Some synonyms require transformations. For example, in order to convert from "log.roll.hours" to
"segment.ms", we must convert hours to milliseconds. (Note that our assumption right now is that
topic configurations do not have synonyms, only broker configurations. If this changes, we will
need to add some logic to handle it.)
This PR makes the 8-argument constructor for ConfigEntry public. We need this in order to make full
use of ConfigEntry outside of the admin namespace. This change is probably inevitable in general
since otherwise we cannot easily test the output from various admin APIs in junit tests outside the
admin package.
Testing:
This PR adds PlaintextAdminIntegrationTest#testCreateTopicsReturnsConfigs. This test validates
some of the configurations that it gets back from the call to CreateTopics, rather than just checking
if it got back a non-empty map like some of the existing tests. In order to test the
configuration override logic, testCreateDeleteTopics now sets up some custom static and dynamic
configurations.
In QuorumTestHarness, we now allow tests to configure what the ID of the controller should be. This
allows us to set dynamic configurations for the controller in testCreateDeleteTopics. We will have
a more complete fix for setting dynamic configuations on the controller later.
This PR changes ConfigurationControlManager so that it is created via a Builder. This will make it
easier to add more parameters to its constructor without having to update every piece of test code
that uses it. It will also make the test code easier to read.
Reviewers: David Arthur <mumrah@gmail.com>
With major server components like the new quorum controller being moved outside of the `core` module, it is useful to have shared dependencies moved into `server-common`. An example of this is Yammer metrics which server components still rely heavily upon. All server components should have access to the default registry used by the broker so that new metrics can be registered and metric naming conventions should be standardized. This is particularly important in KRaft where we are attempting to recreate identically named metrics in the controller context. This patch takes a step in this direction. It moves `KafkaYammerMetrics` into `server-common` and it implements
standard metric naming utilities there.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Show the LeaderRecoveryState in MetadataShell.
Fix a case where we were comparing a Byte type with an enum type.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Implement auto leader rebalance for KRaft by keeping track of the set of topic partitions which have a leader that is not the preferred replica. If this set is non-empty then schedule a leader balance event for the replica control manager.
When applying PartitionRecords and PartitionChangeRecords to the ReplicationControlManager, if the elected leader is not the preferred replica then remember this topic partition in the set of imbalancedPartitions.
Anytime the quorum controller processes a ControllerWriteEvent it schedules a rebalance operation if the there are no pending rebalance operations, the feature is enabled and there are imbalance partitions.
This KRaft implementation only supports the configurations properties auto.leader.rebalance.enable and leader.imbalance.check.interval.seconds. The configuration property leader.imbalance.per.broker.percentage is not supported and ignored.
Reviewers: Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>
Implementation of the protocol for starting and stopping leader recovery after an unclean leader election. This includes the management of state in the controllers (legacy and KRaft) and propagating this information to the brokers. This change doesn't implement log recovery after an unclean leader election.
Protocol Changes
================
For the topic partition state znode, the new field "leader_recovery_state" was added. If the field is missing the value is assumed to be RECOVERED.
ALTER_PARTITION was renamed from ALTER_ISR. The CurrentIsrVersion field was renamed to PartitionEpoch. The new field LeaderRecoveryState was added.
The new field LeaderRecoverState was added to the LEADER_AND_ISR request. The inter broker protocol version is used to determine which version to send to the brokers.
A new tagged field for LeaderRecoveryState was added to both the PartitionRecord and PartitionChangeRecord.
Controller
==========
For both the KRaft and legacy controller the LeaderRecoveryState is set to RECOVERING, if the leader was elected out of the ISR, also known as unclean leader election. The controller sets the state back to RECOVERED after receiving an ALTER_PARTITION request with version 0, or with version 1 and with the LeaderRecoveryState set to RECOVERED.
Both controllers preserve the leader recovery state even if the unclean leader goes offline and comes back online before an RECOVERED ALTER_PARTITION is sent.
The controllers reply with INVALID_REQUEST if the ALTER_PARTITION either:
1. Attempts to increase the ISR while the partition is still RECOVERING
2. Attempts to change the leader recovery state to RECOVERING from a RECOVERED state.
Topic Partition Leader
======================
The topic partition leader doesn't implement any log recovery in this change. The topic partition leader immediately marks the partition as RECOVERED and sends that state in the next ALTER_PARTITION request.
Reviewers: Jason Gustafson <jason@confluent.io>
ConfigurationControl methods should take a boolean indicating whether the resource is newly
created, rather than taking an existence checker object. The boolean is easier to understand. Also
add a unit test of existing checking failing (and succeeding).
Reviewers: Kirk True <kirk@mustardgrain.com>, José Armando García Sancio <jsancio@users.noreply.github.com>
Create KafkaConfigSchema to encapsulate the concept of determining the types of configuration keys.
This is useful in the controller because we can't import KafkaConfig, which is part of core. Also
introduce the TimelineObject class, which is a more generic version of TimelineInteger /
TimelineLong.
Reviewers: David Arthur <mumrah@gmail.com>