The PR removes dependency of server module on share-coordinator, rather it should be other way. Moved the ShareCoordinatorConfig class from server to share-coordinator.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
There are times when the controller has a high event processing time, such as during startup, or when creating a topic with many partitions. We can see these processing times in the p99 metric (kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs), however it's difficult to see exactly which event is causing high processing time.
With DEBUG logs, we see every event along with its processing time. Even with this, it's a bit tedious to find the event with a high processing time.
This PR logs all events which take longer than 2 seconds at ERROR level. This will help identify events that are taking far too long, and which could be disruptive to the operation of the controller. The slow event logging looks like this:
```
[2024-12-20 15:03:39,754] ERROR [QuorumController id=1] Exceptionally slow controller event createTopics took 5240 ms. (org.apache.kafka.controller.EventPerformanceMonitor)
```
Also, every 60 seconds, it logs some event time statistics, including average time, maximum time, and the name of the event which took the longest. This periodic message looks like this:
```
[2024-12-20 15:35:04,798] INFO [QuorumController id=1] In the last 60000 ms period, 333 events were completed, which took an average of 12.34 ms each. The slowest event was handleCommit[baseOffset=0], which took 41.90 ms. (org.apache.kafka.controller.EventPerformanceMonitor)
```
An operator can disable these logs by adding the following to their log4j config:
```
org.apache.kafka.controller.EventPerformanceMonitor=OFF
```
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Relevant methods:
1. `List.of`, `Set.of`, `Map.of` and similar (introduced in Java 9)
2. Optional: `isEmpty` (introduced in Java 11), `stream` (introduced in Java 9).
Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
This makes it possible to enable request logs for deprecated protocol api versions without enabling it for the rest. Combined with the ability to enable/disable dynamically, it makes it a bit easier to collect the information about deprecated clients that is not available via metrics.
This isn't particularly useful in trunk/4.0 since there are no deprecated api versions in these versions, but it will be useful for older branches. I intend to backport to those branches and add a release note in the backport regarding the change in behavior.
I manually verified that:
1. If the request logger is configured at `INFO` level, only deprecated protocol api versions are logged and they are logged at `INFO` level.
2. If the request logger is configured at `DEBUG` level, all requests are logged but the log level is `INFO` for deprecated protocol api versions and `DEBUG` for the rest.
3. If the request logger is configured at `WARN` level (the default), no requests are logged.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Adds a new RPC StreamsGroupDescribe that returns, given the group ID, all metadata related to the streams group, such as
- The topology metadata of the group.
- The topology epoch of the group.
- The latest member metadata that each member provided through the StreamsGroupHeartbeat API.
- The current target assignment generated by the assignor.
- This just adds the JSON as defined in KIP-1071, together with some plumbing.
Reviewers: Bill Bejeck <bbejeck@gmail.com>
The StreamsGroupHeartbeat API is the new core API used by streams application to form a group. The API allows members to initialize a topology, advertise their state, and their owned tasks. The group coordinator uses it to assign/revoke tasks to/from members. This API is also used as a liveness check.
This change adds the JSON definition of the RPC, as defined in KIP-1071.
Reviewers: Bruno Cadonna <cadonna@apache.org>
In this PR, we've added a class ShareCoordinatorOffsetsManager, which tracks the last redundant offset for each share group state topic partition. We have also added a periodic timer job in ShareCoordinatorService which queries for the redundant offset at regular intervals and if a valid value is found, issues the deleteRecords call to the ReplicaManager via the PartitionWriter. In this way the size of the partitions is kept manageable.
Reviewers: Jun Rao <junrao@gmail.com>, David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>
Currently the validation of feature upgrade relies on the supported version range generated during registration. For a given feature, its max supported feature version in production is set to be the default version value (the latest feature version with bootstrap metadata value smaller or equal to the latest production metadata value).
This patch introduces a LATEST_PRODUCTION value independent from the metadata version to each feature so that the highest supported feature version can be customized by the feature owner.
The change only applies to dynamic feature upgrade. During formatting, we still use the default value associated the metadata version.
Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
The ClientsMetricManager keeps the client instance cache at a limit of 16384. The active connections can be low, but connections can be created and destroyed in short span. Which hits the cache limit and results in cache miss for new connections.
The client instance cache keeps the instances for 3 * push interval ms. Hence when the cache gets full it creates new instances for every request while evicting others from cache. This gives some bumps to the GC for the broker and eventually CPU. Though with evicting early it will still be garbage collected but the long running active connections will not be removed from the cache.
This PR adds capability to evict client instances from cache when the connection from client is dropped.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
When MetadataBatchLoader handles a BeginTransactionRecord, it will publish the metadata that has seen so far and not publish again until the transaction is ended or aborted. This means a partial record batch can be published. If a snapshot is generated during this time, the currently published metadata may not align with the end of a record batch. This causes problems with Raft replication which expects a snapshot's offset to exactly precede a record batch boundary.
This patch enhances SnapshotGenerator to refuse to generate a snapshot if the metadata is not batch aligned.
Reviewers: David Arthur <mumrah@gmail.com>
This PR is the first series in the attempt to add share.record.lock.duration.ms to dynamic group configs. As part of this PR, the ShareGroupConfig has been moved to org.apache.kafka.coordinator.group.modern.share
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Abhinav Dixit <adixit@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
This fixes some formatting issues with the control.plane.listener.format.name property. It was missing some new lines and code markup.
For testing, I built locally and viewed the output.
Reviewers: Justine Olshan <jolshan@confluent.io>
Introduced a share fetch purgatory on the broker which delays share fetch requests that cannot be completed instantaneously. Introduced 2 new classes -
DelayedShareFetch - Contains logic to instantaneously complete or force complete a share fetch request on timeout.
DelayedShareFetchKey - Contains the key which can be used to watch the entries within the share fetch purgatory.
ShareFetchUtils - This utility class contains functionalities required for post-processing once the replica manager fetch is completed.
There are many scenarios which can cause a share fetch request to be delayed and multiple scenarios when a delayed share fetch can be attempted to be completed. In this PR, we are only targeting the case when record lock partition limit is reached, the ShareFetch should wait for up to MaxWaitMs for records to be released.
Reviewers: David Arthur <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
Introduces the share coordinator. This coordinator is built on the new coordinator runtime framework. It
is responsible for persistence of share-group state in a new internal topic named "__share_group_state".
The responsibility for being a share coordinator is distributed across the brokers in a cluster.
Reviewers: David Arthur <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
Previously in KRaft mode, we could request an unclean leader election for a specific topic using
the electLeaders API. This PR adds an additional way to trigger unclean leader election when in
KRaft mode via the static controller configuration and various dynamic configurations.
In order to support all possible configuration methods, we have to do a multi-step configuration
lookup process:
1. check the dynamic topic configuration for the topic.
2. check the dynamic node configuration.
3. check the dynamic cluster configuration.
4. check the controller's static configuration.
Fortunately, we already have the logic to do this multi-step lookup in KafkaConfigSchema.java.
This PR reuses that logic. It also makes setting a configuration schema in
ConfigurationControlManager mandatory. Previously, it was optional for unit tests.
Of course, the dynamic configuration can change over time, or the active controller can change
to a different one with a different configuration. These changes can make unclean leader
elections possible for partitions that they were not previously possible for. In order to address
this, I added a periodic background task which scans leaderless partitions to check if they are
eligible for an unclean leader election.
Finally, this PR adds the UncleanLeaderElectionsPerSec metric.
Co-authored-by: Luke Chen showuon@gmail.com
Reviewers: Igor Soarez <soarez@apple.com>, Luke Chen <showuon@gmail.com>
Implement the remaining ForwardingManager metrics from KIP-938: Add more metrics for measuring KRaft performance:
kafka.server:type=ForwardingManager,name=QueueTimeMs.p99
kafka.server:type=ForwardingManager,name=QueueTimeMs.p999
kafka.server:type=ForwardingManager,name=QueueLength
kafka.server:type=ForwardingManager,name=RemoteTimeMs.p99
kafka.server:type=ForwardingManager,name=RemoteTimeMs.p999
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Establishes the new `:share` Gradle module. This module is intended to be used for server-side KIP-932 classes that are not part of the new share group coordinator.
This patch relocates and renames some existing classes. A small amount of compatibility changes were also made, but do not affect any logic.
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur <mumrah@gmail.com>
This patch extends the DescribeConfigs API to support group configs.
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Jacot <djacot@confluent.io>
Add the "share" group coordinator rebalance protocol as the way to enable KIP-932. It is also necessary to turn on the new group coordinator.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Implemented handleShareAcknowledge request RPC in KafkaApis.scala. This method is called whenever the client sends a Share Acknowledge request to the broker. The acknowledge logic is handles asynchronously and the results are handled appropriately.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
In MetadataVersion 3.7-IV2 and above, the broker's AssignmentsManager sends an RPC to the
controller informing it about which directory we have chosen to place each new replica on.
Unfortunately, the code does not check to see if the topic still exists in the MetadataImage before
sending the RPC. It will also retry infinitely. Therefore, after a topic is created and deleted in
rapid succession, we can get stuck including the now-defunct replica in our subsequent
AssignReplicasToDirsRequests forever.
In order to prevent this problem, the AssignmentsManager should check if a topic still exists (and
is still present on the broker in question) before sending the RPC. In order to prevent log spam,
we should not log any error messages until several minutes have gone past without success.
Finally, rather than creating a new EventQueue event for each assignment request, we should simply
modify a shared data structure and schedule a deferred event to send the accumulated RPCs. This
will improve efficiency.
Reviewers: Igor Soarez <i@soarez.me>, Ron Dagostino <rndgstn@gmail.com>
This PR adds support for add-controller and remove-controller in the kafka-metadata-quorum.sh
command-line tool. It also fixes some minor server-side bugs that blocked the tool from working.
In kafka-metadata-quorum.sh, the implementation of remove-controller is fairly straightforward. It
just takes some command-line flags and uses them to invoke AdminClient. The add-controller
implementation is a bit more complex because we have to look at the new controller's configuration
file. The parsing logic for the advertised.listeners and listeners server configurations that we
need was previously implemented in the :core module. However, the gradle module where
kafka-metadata-quorum.sh lives, :tools, cannot depend on :core. Therefore, I moved listener parsing
into SocketServerConfigs.listenerListToEndPoints. This will be a small step forward in our efforts
to move Kafka configuration out of :core.
I also made some minor changes in kafka-metadata-quorum.sh and Kafka-storage-tool.sh to handle
--help without displaying a backtrace on the screen, and give slightly better error messages on
stderr. Also, in DynamicVoter.toString, we now enclose the host in brackets if it contains a colon
(as IPV6 addresses can).
This PR fixes our handling of clusterId in addRaftVoter and removeRaftVoter, in two ways. Firstly,
it marks clusterId as nullable in the AddRaftVoterRequest.json and RemoveRaftVoterRequest.json
schemas, as it was always intended to be. Secondly, it allows AdminClient to optionally send
clusterId, by using AddRaftVoterOptions and RemoveRaftVoterOptions. We now also remember to
properly set timeoutMs in AddRaftVoterRequest. This PR adds unit tests for
KafkaAdminClient#addRaftVoter and KafkaAdminClient#removeRaftVoter, to make sure they are sending
the right things.
Finally, I fixed some minor server-side bugs that were blocking the handling of these RPCs.
Firstly, ApiKeys.ADD_RAFT_VOTER and ApiKeys.REMOVE_RAFT_VOTER are now marked as forwardable so that
forwarding from the broker to the active controller works correctly. Secondly,
org.apache.kafka.raft.KafkaNetworkChannel has now been updated to enable API_VERSIONS_REQUEST and
API_VERSIONS_RESPONSE.
Co-authored-by: Murali Basani muralidhar.basani@aiven.io
Reviewers: José Armando García Sancio <jsancio@apache.org>, Alyssa Huang <ahuang@confluent.io>
ShareGroupHeartbeat API support as defined in KIP-932. The heartbeat persists Group and Member information on __consumer_offsets topic.
The PR also moves some of the ShareGroupConfigs to GroupCoordinatorConfigs as they should only be used in group coordinator.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
The release API exposed Partitions which should be an internal implementation detail for releaseAcquiredRecords API. Also lessen the scope for cached topic partitions method as it's not needed.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>, Abhinav Dixit <adixit@confluent.io>
A few of the share group configs in KIP-932 were defined with limits that do not match KIP-932. This PR corrects the limits.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
About
KIP-932 introduces share sessions for share groups. This PR implements share sessions and contexts for incoming share fetch requests on broker. The changes include:
Defined CachedSharePartition class which are stored in share sessions.
Defined ShareSessionKey, ShareSession classes.
Defined ShareSessionCache class which caches all the share sessions and has evict policy defined as per KIP-932
Defined the 2 types of contexts -
a. ShareSessionContext - for share session fetch request.
b. FinalContext - for final share fetch request (epoch = -1).
Defined newContext function which returns the created/updated context on receiving share fetch request on broker.
Testing
The added code has been tested with the help of unit tests present in the PR.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
The implementation for share-fetch next-fetch-offset in share partition and acquiring records from log.
The Next Fetch Offset (NFO) determines where the Share Partition should initiate the next data read from the Replica Manager. While it typically aligns with the last offset of the most recently returned batch, last offset + 1, there are exceptions. Messages marked available again due to release acknowledgements or lock timeouts can cause the NFO to shift.
The acquire method caches the batches as acquired in-memory and spawns a timer task for lock timeout.
Cache
Per-offset Metadata: Simple to implement but inefficient. Every offset requires in-memory storage and traversal, leading to high memory usage and processing overhead, especially for per-batch acknowledgements (mostly the way records would be acknowledged).
Per-Replica Fetch Batch: This approach aligns with the Replica Manager fetch batches. Since a full Replica Manager batch is retrieved whenever the requested offset falls within that batch's boundaries, a single Share Fetch request will likely receive an entire Replica Manager batch. However, there's a trade-off. Replica Manager batches are based on producer batching. If producers don't batch effectively, the in-flight metadata becomes heavily reliant on the producer's batching behavior.
For per-message acknowledgements, per-offset tracking will be necessary which again requires splitting in-flight batches based on state. Splitting bacthes is inefficient as it requires cache update wshich maintains sorted order. Therefore, we propose a hybrid approach:
Implemented a combination of option 2 (per-in-flight batch tracking) with option 1 (per-offset tracking). This aligns well with Replica Manager batching.
States shall be maintained per in-flight batch. If state inconsistencies arise within in-flight batches due to per-message acknowledgements, switch state tracking for the respective batch to option 1 (per-offset tracking).
Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>, Abhinav Dixit <144765188+adixitconfluent@users.noreply.github.com>
KIP-932 introduces a bunch of broker and dynamic configs for share groups. This PR adds those new configs. The changes include:
1. Defined ShareGroupConfigs class which stores various share group configurations.
2. Use the defined share configs in KafkaConfig.scala for making it available to BrokerServer
3. Adds a few tests to validate the conditions on these new configs.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
JBOD Brokers keep the Controller up to date with replica-to-directory
placement via AssignReplicasToDirsRequest. These requests are queued,
compacted and sent by AssignmentsManager.
The Controller returns the error NOT_LEADER_OR_FOLLOWER when handling
a AssignReplicasToDirsRequest from a broker that is not a replica.
A partition reassignment can take place, removing the Broker
as a replica before the AssignReplicasToDirsRequest successfully
reaches the Controller. AssignmentsManager retries failed
requests, and will continuously try to propagate this assignment,
until the Broker either shuts down, or is added back as a replica.
When encountering a NOT_LEADER_OR_FOLLOWER error, AssignmentsManager
should assume that the broker is no longer a replica, and stop
trying to propagate the directory assignment for that partition.
Reviewers: Luke Chen <showuon@gmail.com>
As per KIP-1022, we will rename the unstable metadata versions enabled config to support all feature versions.
Features is also updated to return latest production and latest testing versions of each feature.
A feature is production ready when the corresponding metadata version (bootstrapMetadataVersion) is production ready.
Adds tests for the feature usage of the unstableFeatureVersionsEnabled config
Reviewers: David Jacot <djacot@confluent.io>, Jun Rao <junrao@gmail.com>
At the moment it can be a bit difficult to troubleshoot issues related to the AssignmentsManager. Mainly because:
Topic partitions are logged with topic ID and partition index but without the topic name.
Directory IDs are logged without the directory path.
Assignment reasons aren't tracked.
This patch addresses the three issues.
Reviewers: Luke Chen <showuon@gmail.com>
This patch adds the `group.consumer.migration.policy` config which controls how consumer groups can be converted from classic group to consumer group and vice versa. The config is kept as an internal one while we develop the feature.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Jacot <djacot@confluent.io>
Updates the broker metrics name to kebab/hyphen case. Also removed the redundant client-metrics- prefix in all metrics as the group name in client-metrics itself.
Reviewers: David Jacot <djacot@confluent.io>, Jun Rao <junrao@gmail.com>
After profiling the kafka tests, tons of client-metrics-reaper thread not cleanup after BrokerServer shutdown.
The thread client-metrics-reaper comes from ClientMetricsManager#expirationTimer, and BrokerServer#shudown doesn't close ClientMetricsManager which let the thread still runs in background.
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
When moving replicas between directories in the same broker, future replica promotion hinges on acknowledgment from the controller of a change in the directory assignment.
ReplicaAlterLogDirsThread relies on AssignmentsManager for a completion notification of the directory assignment change.
In its current form, under certain assignment scheduling, AssignmentsManager both miss completion notifications, or prematurely trigger them.
Reviewers: Luke Chen <showuon@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Gaurav Narula <gaurav_narula2@apple.com>
1) This PR moves kafka.security classes from core to server module.
2) AclAuthorizer not moved, because it has heavy dependencies on core classes that not rewrited from scala at the moment.
3) AclAuthorizer will be deleted as part of ZK removal
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
Co-authored-by: n.izhikov <n.izhikov@vk.team>
In order to move ConfigCommand to tools we must move all it's dependencies which includes KafkaConfig and other core classes to java. This PR moves log cleaner configuration to CleanerConfig class of storage module.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
- Removing ! and Unused Imports
- Put a space after the control structure's defining keyword.
- remove unnecessary whitespace a space after the method name in higher-order function invocations.
Reviewers: Divij Vaidya <diviv@amazon.com>
During migration from ZK mode to KRaft mode, there is a step where the kcontrollers load all of the
data from ZK into the metadata log. Previously, we were using a batch size of 1000 for this, but
200 seems better. This PR also adds an internal configuration to control this batch size, for
testing purposes.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
The uniform assignor got accidentally removed by https://github.com/apache/kafka/pull/15158. This patch adds it back.
Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
This patch implements the new DescribeTopicPartitions RPC as defined in KIP-966 (ELR). Additionally, this patch adds a broker config "max.request.partition.size.limit" which limits the number of partitions returned by the new RPC.
Reviewers: Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>, David Arthur <mumrah@gmail.com>
KIP-714 requires client instance cache in broker which should also have a time-based eviction policy where client instances which are not actively sending metrics should be evicted. KIP mentions This client instance specific state is maintained in broker memory up to MAX(60*1000, PushIntervalMs * 3) milliseconds.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk> , David Jacot <djacot@confluent.io>, Nikolay <NIzhikov@gmail.com>
PR adds support to capture client socket port information in Request Context. The port from request context is used as matching criteria in filtering clients and shall be used by metrics plugin to fetch port from request context.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
- Add proper start & stop for AssignmentsManager's event loop
- Dedupe queued duplicate assignments
- Fix bug where directory ID is resolved too late
Co-authored-by: Gaurav Narula <gaurav_narula2@apple.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Handle AssignReplicasToDirs requests, persist metadata changes
with new directory assignments and possible leader elections.
Reviewers: Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rndgstn@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Improvement for KIP-1000 to list client metrics resources in KafkaApis.scala. Using functionality exposed by KIP-1000 to support describe all metrics operations for KIP-714.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
The PR adds handling of telemetry APIs in KafkaAPIs.scala which calls the respective manager to handle the API calls. Also the telemetry plugin which if registered in configs get registered for exporting client metrics.
Reviewers: Jun Rao <junrao@gmail.com>
The PR provide implementation for client metrics manager along with other classes. Manager is responsible to support 3 operations:
UpdateSubscription - From kafka-configs.sh and reload from metadata cache.
Process Get Telemetry Request - From KafkaApis.scala
Process Push Telemetry Request - From KafkaApis.scala
Manager maintains an in-memory cache to keep track of client instances against their instance id.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
Building AssignReplicasToDirsRequestData relies on iteration over Map entries, which can result in different sorting order. The order does not matter to the semantics of the request, but it can cause issues with test assertions. This issue was introduced in #14369.
Reviewers: Divij Vaidya <diviv@amazon.com>, Ismael Juma <ismael@juma.me.uk>, David Jacot <djacot@confluent.io>
We only move Java classes that have minimal or no dependencies on Scala classes in this PR.
Details:
* Configured `server` module in build files.
* Changed `ControllerRequestCompletionHandler` to be an interface since it has no implementations.
* Cleaned up various import control files.
* Minor build clean-ups for `server-common`.
* Disabled `testAssignmentAggregation` when executed with Java 8, this is an existing issue (see #14794).
For broader context on this change, please check:
* KAFKA-15852: Move server code from `core` to `server` module
Reviewers: Divij Vaidya <diviv@amazon.com>