In SharedServer, fix some cases where a volatile variable could change to null while we were using
it, during shutdown. This is mainly a junit test issue, although it could also cause ugly error
messages during shutdown when running the server in a production context.
Fix a race in KafkaEventQueueTest.testSize.
Reviewers: David Arthur <mumrah@gmail.com>
Added functionality to copy log segments, indexes to the target remote storage for each topic partition enabled with tiered storage. This involves creating scheduled tasks for all leader partition replicas to copy their log segments in sequence to tiered storage.
Reviewers: Jun Rao <junrao@gmail.com>, Luke Chen <showuon@gmail.com>
The motivation for introducing InMemoryLeaderEpochCheckpoint is to allow remote log manager to create the RemoteLogSegmentMetadata(RLSM) with the correct leader epoch info for a specific segment. To do that, we need to rely on the LeaderEpochCheckpointCache to truncate from start and end, to get the epoch info. However, we don't really want to truncate the epochs in cache (and write to checkpoint file in the end). So, we introduce this InMemoryLeaderEpochCheckpoint to feed into LeaderEpochCheckpointCache, and when we truncate the epoch for RLSM, we can do them in memory without affecting the checkpoint file, and without interacting with file system.
Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
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>
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 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>
* KAFKA-14524: Rewrite KafkaMetricsGroup in Java
Instead of being a base trait for classes, `KafkaMetricsGroup` is now an independent object. User classes could override methods in it to adjust its behavior like they used to with the trait model.
Some classes were extending the `KafkaMetricsGroup` trait, but it wasn't actually used.
Reviewers: Ismael Juma <ismael@juma.me.uk>, lbownik <lukasz.bownik@gmail.com>, Satish Duggana <satishd@pache.org>
This patch adds ConsumerGroupHeartbeat to the GroupCoordinator interface and implements the API in KafkaApis.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
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>
* 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>
Extend the implementation of ProcessTerminatingFaultHandler to support calling either Exit.halt or Exit.exit. Change the fault handler used by the Controller thread and the KRaft thread to use a halting fault handler.
Those threads cannot call Exit.exit because Runtime.exit joins on the default shutdown hook thread. The shutdown hook thread joins on the controller and kraft thread terminating. This causes a deadlock.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
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 we are listening on fixed ports, we should defer opening ports until we're ready to accept
traffic. If we open the broker port too early, it can confuse monitoring and deployment systems.
This is a particular concern when in KRaft mode, since in that mode, we create the SocketServer
object earlier in the startup process than when in ZK mode.
The approach taken in this PR is to defer opening the acceptor port until Acceptor.start is called.
Note that when we are listening on a random port, we continue to open the port "early," in the
SocketServer constructor. The reason for doing this is that there is no other way to find the
random port number the kernel has selected. Since random port assignment is not used in production
deployments, this should be reasonable.
FutureUtils.java: add chainFuture and tests.
SocketServerTest.scala: add timeouts to cases where we call get() on futures.
Reviewers: David Arthur <mumrah@gmail.com>, Alexandre Dupriez <hangleton@users.noreply.github.com>
When running junit tests, it is not good to block forever on CompletableFuture objects. When there
are bugs, this can lead to junit tests hanging forever. Jenkins does not deal with this well -- it
often brings down the whole multi-hour test run. Therefore, when running integration tests in
JUnit, set some reasonable time limits on broker and controller startup time.
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>
There were some concurrency inconsistencies in `KafkaScheduler` flagged by spotBugs
that had to be fixed, summary of changes below:
* Executor is `volatile`
* We always synchronize and check `isStarted` as the first thing within the critical
section when a mutating operation is performed.
* We don't synchronize (but ensure the executor is not null in a safe way) in read-only
operations that operate on the executor.
With regards to `MockScheduler/MockTask`:
* Set the type of `nextExecution` to `AtomicLong` and replaced inconsistent synchronization
* Extracted logic into `MockTask.rescheduleIfPeriodic`
Tweaked the `Scheduler` interface a bit:
* Removed `unit` parameter since we always used `ms` except one invocation
* Introduced a couple of `scheduleOnce` overloads to replace the usage of default
arguments in Scala
* Pulled up `resizeThreadPool` to the interface and removed `isStarted` from the
interface.
Other cleanups:
* Removed spotBugs exclusion affecting `kafka.log.LogConfig`, which no longer exists.
For broader context, see:
* KAFKA-14470: Move log layer to storage module
Reviewers: Jun Rao <junrao@gmail.com>
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>
Escape the `>` character in javadoc
Escape the `$` character when part of `${}` in scaladoc as this is the way to reference a variable
Reviewers: Matthias J. Sax <matthias@confluent.io>
This is a requirement for:
* KAFKA-14477: Move LogValidator to storage module.
For broader context on this change, please check:
* KAFKA-14470: Move log layer to storage module
Reviewers: dengziming <dengziming1993@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>
Add comments to clarify that both offsets and partitions are 0-indexed, and fix a minor typo. Clarify which offset will be retrieved by poll() after seek() is used in various circumstances. Also added integration tests.
Reviewers: Luke Chen <showuon@gmail.com>
This PR enables brokers which are upgrading from ZK mode to KRaft mode to forward certain metadata
change requests to the controller instead of applying them directly through ZK. To faciliate this,
we now support EnvelopeRequest on zkBrokers (instead of only on KRaft nodes.)
In BrokerToControllerChannelManager, we can now reinitialize our NetworkClient. This is needed to
handle the case when we transition from forwarding requests to a ZK-based broker over the
inter-broker listener, to forwarding requests to a quorum node over the controller listener.
In MetadataCache.scala, distinguish between KRaft and ZK controller nodes with a new type,
CachedControllerId.
In LeaderAndIsrRequest, StopReplicaRequest, and UpdateMetadataRequest, switch from sending both a
zk and a KRaft controller ID to sending a single controller ID plus a boolean to express whether it
is KRaft. The previous scheme was ambiguous as to whether the system was in KRaft or ZK mode when
both IDs were -1 (although this case is unlikely to come up in practice). The new scheme avoids
this ambiguity and is simpler to understand.
Reviewers: dengziming <dengziming1993@gmail.com>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
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>
`core` should only be used for legacy cli tools and tools that require
access to `core` classes instead of communicating via the kafka protocol
(typically by using the client classes).
Summary of changes:
1. Convert the command implementation and tests to Java and move it to
the `tools` module.
2. Introduce mechanism to capture stdout and stderr from tests.
3. Change `kafka-metadata-quorum.sh` to point to the new command class.
4. Adjusted the test classpath of the `tools` module so that it supports tests
that rely on the `@ClusterTests` annotation.
5. Improved error handling when an exception different from `TerseFailure` is
thrown.
6. Changed `ToolsUtils` to avoid usage of arrays in favor of `List`.
Reviewers: dengziming <dengziming1993@gmail.com>
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>
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 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>
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>
Add `MetadataQuorumCommand` to describe quorum status, I'm trying to use arg4j style command format, currently, we only support one sub-command which is "describe" and we can specify 2 arguments which are --status and --replication.
```
# describe quorum status
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --replication
ReplicaId LogEndOffset Lag LastFetchTimeMs LastCaughtUpTimeMs Status
0 10 0 -1 -1 Leader
1 10 0 -1 -1 Follower
2 10 0 -1 -1 Follower
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --status
ClusterId: fMCL8kv1SWm87L_Md-I2hg
LeaderId: 3002
LeaderEpoch: 2
HighWatermark: 10
MaxFollowerLag: 0
MaxFollowerLagTimeMs: -1
CurrentVoters: [3000,3001,3002]
CurrentObservers: [0,1,2]
# specify AdminClient properties
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 --command-config config.properties describe --status
```
Reviewers: Jason Gustafson <jason@confluent.io>
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>
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>
* 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>
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>
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 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>
Add a DeleteTopics test for all supported versions. Convert the
DeleteTopicsRequestTest to run against both ZK and KRaft mode.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>
These tests belongs to ApiVersionsResponseTest, and accidentally copied them to MetadataVersionTest when working on #12072.
Reviewer: Luke Chen <showuon@gmail.com>
Refactoring ApiVersion to MetadataVersion to support both old IBP versioning and new KRaft versioning (feature flags)
for KIP-778.
IBP versions are now encoded as enum constants and explicitly prefixed w/ IBP_ instead of KAFKA_, and having a
LegacyApiVersion vs DefaultApiVersion was not necessary and replaced with appropriate parsing rules for extracting
the correct shortVersions/versions.
Co-authored-by: David Arthur <mumrah@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
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>
The current naming of the fields in `ProducerIdsRecord` is a little confusing in regard to whether the block range was inclusive or exclusive. This patch tries to improve naming to make this clearer. In the record class, instead of `ProducerIdsEnd`, we use `NextProducerId`. We have also updated related classes such as `ProducerIdsBlock.java` with similar changes.
Reviewers: dengziming <dengziming1993@gmail.com>, David Arthur <mumrah@gmail.com>
Avoid O(N) behavior in KRaftMetadataCache#topicNamesToIds and
KRaftMetadataCache#topicIdsToNames by returning a map subclass that
exposes the TopicsImage data structures without copying them.
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
MINOR Refactored the existing CheckpointFile in core module, moved to server-common module.
Refactored CheckpointFile to server-common module as a Java class and it is reused by LeaderCheckpointFile, OffsetCheckpointFile.
This will be used by CommittedOffsetsFile which checkpoints remote log metadata partitions with respective offsets in the default RemoteLogMetadataManager implementation.
Existing tests are available for LeaderCheckpointFile, OffsetCheckpointFile.
Reviewers: Jun Rao <junrao@gmail.com>
This patch bumps the default frame version for kraft records from 0 to 1. At the same time, we reset all
records versions back to 0 and we enable flexible version support for UnregisterBrokerRecord, which was
missed previously. Note that the frame version bump also affects the KIP-405 records since they are
sharing AbstractApiMessageSerde. Since these records were not part of any previous releases, this should
not cause an issue.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
`MetadataParser` is a duplication of `MetadataRecordSerde` and it's not used in any code, so we can remove it. It did, however, have some useful validations which have been moved into `MetadataRecordSerde`.
Reviewers: Jason Gustafson <jason@confluent.io>
Since KafkaEventQueue is a generic data structure not specific to metadata, move it
into the server-common module.
Reviewers: Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>
Introduce new AllocateProducerIds RPC and IBP 3.0-IV0 as part of KIP-730.
This change adds a new AllocateProducerIds RPC which is used by the broker to request a block of
producer IDs from the controller. The new IBP added will determine if the broker should talk directly to
ZooKeeper (IBP < 3.0) or it if should use the new RPC to talk to the controller (IBP >= 3.0).
Per-broker property overrides for ClusterTests were also added (in order to test mixed IBPs in a cluster)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Added server-common module to have server side common classes. Moved ApiMessageAndVersion, RecordSerde, AbstractApiMessageSerde, and BytesApiMessageSerde to server-common module.
Reivewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>