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>