A patch for KAFKA-15046 got rid of fsync on LeaderEpochFileCache#truncateFromStart/End for performance reason, but it turned out this could cause corrupted leader-epoch checkpoint file on ungraceful OS shutdown, i.e. OS shuts down in the middle when kernel is writing dirty pages back to the device.
To address this problem, this PR makes below changes: (1) Revert LeaderEpochCheckpoint#write to always fsync
(2) truncateFromStart/End now call LeaderEpochCheckpoint#write asynchronously on scheduler thread
(3) UnifiedLog#maybeCreateLeaderEpochCache now loads epoch entries from checkpoint file only when current cache is absent
Reviewers: Jun Rao <junrao@gmail.com>
This patch fixes a typo in MetadataVersion.IBP_4_0_IV0. It should be 0 not O.
Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This patch introduces the `group.version` feature flag with one version:
1) Version 1 enables the new consumer group rebalance protocol (KIP-848).
Reviewers: Justine Olshan <jolshan@confluent.io>
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>
As part of KIP-1022, I have created an interface for all the new features to be used when parsing the command line arguments, doing validations, getting default versions, etc.
I've also added the --feature flag to the storage tool to show how it will be used.
Created a TestFeatureVersion to show an implementation of the interface (besides MetadataVersion which is unique) and added tests using this new test feature.
I will add the unstable config and tests in a followup.
Reviewers: David Mao <dmao@confluent.io>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jun Rao <junrao@apache.org>
A broker that is unable to communicate with the controller will shut down
after the configurable log.dir.failure.timeout.ms.
The implementation adds a new event to the Kafka EventQueue. This event
is deferred by the configured timeout and will execute the shutdown
if the heartbeat communication containing the failed log dir is still
pending with the controller.
Reviewers: Igor Soarez <soarez@apple.com>
Write events create and add a TimerTask to schedule the timeout operation. The issue is that we pile up the number of timer tasks which are essentially no-ops if replication was successful. They stay in memory for 15 seconds (default write timeout) and as the rate of write increases, the impact on memory usage increases.
Instead, cancel the corresponding write timeout task when the write event is committed to the log. This also applies to complete transaction events.
Reviewers: David Jacot <djacot@confluent.io>
We observe some thread leaks in CI which point to the executor service
thread. This change tries to shutdown the executor service using the
helper method in `ThreadUtils`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>
Instead of entering pending forever, this PR invoke next schedule after 1ms. However, the side effect is busy-waiting. Hence, This PR also update the docs to remind users about that - the issue about smaller log.segment.delete.delay.ms
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@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>
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>
- Use `Empty` instead of 'none' when referring to `Optional` values.
- `Headers.lastHeader` returns `null` when no header is found.
- Fix minor spelling mistakes.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
KAFKA-16222 fixed a bug whereby we didn't undo the name sanitization used on client quota entity names
stored in ZooKeeper. However, it incorrectly claimed to fix the handling of default client quota
entities. It also failed to correctly re-sanitize when syncronizing the data back to ZooKeeper.
This PR fixes ZkConfigMigrationClient to do the sanitization correctly on both the read and write
paths. We do de-sanitization before invoking the visitors, since after all it does not make sense to
do the same de-sanitization step in each and every visitor.
Additionally, this PR fixes a bug causing default entities to be converted incorrectly. For example,
ClientQuotaEntity(user -> null) is stored under the /config/users/<default> znode in ZooKeeper. In
KRaft it appears as a ClientQuotaRecord with EntityData(entityType=users, entityName=null).
Prior to this PR, this was being converted to a ClientQuotaRecord with EntityData(entityType=users,
entityName=""). That represents a quota on the user whose name is the empty string (yes, we allow
users to name themselves with the empty string, sadly.)
The confusion appears to have arisen because for TOPIC and BROKER configurations, the default
ConfigResource is indeed the one named with the empty (not null) string. For example, the default
topic configuration resource is ConfigResource(name="", type=TOPIC). However, things are different
for client quotas. Default client quota entities in KRaft (and also in AdminClient) are represented
by maps with null values. For example, the default User entity is represented by Map("user" ->
null). In retrospect, using a map with null values was a poor choice; a Map<String,
Optional<String>> would have made more sense. However, this is the way the API currently is and we
have to convert correctly.
There was an additional level of confusion present in KAFKA-16222 where someone thought that using
the ZooKeeper placeholder string "<default>" in the AdminClient API would yield a default client
quota entity. Thise seems to have been suggested by the ConfigEntityName class that was created
recently. In fact, <default> is not part of any public API in Kafka. Accordingly, this PR also
renames ConfigEntityName.DEFAULT to ZooKeeperInternals.DEFAULT_STRING, to make it clear that the
string <default> is just a detail of the ZooKeeper implementation. It is not used in the Kafka API
to indicate defaults. Hopefully this will avoid confusion in the future.
Finally, the PR also creates KRaftClusterTest.testDefaultClientQuotas to get extra test coverage of
setting default client quotas.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Igor Soarez <soarez@apple.com>
There were different words for metadata.version like metadata version or metadataVersion. Unify format as metadata.version.
Reviewers: Luke Chen <showuon@gmail.com>
In between HeartbeatRequest being sent and the response being handled,
i.e. while a HeartbeatRequest is in flight, an extra request may be
immediately scheduled if propagateDirectoryFailure, setReadyToUnfence,
or beginControlledShutdown is called.
To prevent the extra request, we can avoid the extra requests by checking
whether a request is in flight, and delay the scheduling if necessary.
Some of the tests in BrokerLifecycleManagerTest are also improved to
remove race conditions and reduce flakiness.
Reviewers: Colin McCabe <colin@cmccabe.xyz>, Ron Dagostino <rdagostino@confluent.io>, Jun Rao <junrao@gmail.com>
It is possible that due to resource constraint, ShutdownableThread#run might be called later than the ShutdownableThread#close method.
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>
There are a few cases in the group coordinator service where we want to read from or write to each of the known coordinators (each of __consumer_offsets partitions). The current implementation needs to get the list of the known coordinators then schedules the operation and finally aggregate the results. This patch is an attempt to streamline this by adding multi read/write to the runtime.
Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
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>
This patch uniformizes the error handling in the GroupCoordinatorService with the aim to reuse the same error translation for all operations. It also ensures that exceptions are unwrapped if needed.
Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
We update metadata update handler to resend broker registration when
metadata has been updated to >= 3.7IV2 so that the controller becomes
aware of the log directories in the broker.
We also update DirectoryId::isOnline to return true on an empty list of
log directories while the controller awaits broker registration.
Co-authored-by: Proven Provenzano <pprovenzano@confluent.io>
Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
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>
This PR creates MetadataVersion.latestTesting to represent the highest metadata version (which may be unstable) and MetadataVersion.latestProduction to represent the latest version that should be used in production. It fixes a few cases where the broker was advertising that it supported the testing versions even when unstable metadata versions had not been configured.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
Moves ELR from MetadataVersion IBP_3_7_IV3 into the new IBP_3_8_IV0 because the ELR feature was not completed before 3.7 reached feature freeze. Leaves IBP_3_7_IV3 empty -- it is a no-op and is not reused for anything. Adds the new MetadataVersion IBP_3_7_IV4 for the FETCH request changes from KIP-951, which were mistakenly never associated with a MetadataVersion. Updates the LATEST_PRODUCTION MetadataVersion to IBP_3_7_IV4 to declare both KRaft JBOD and the KIP-951 changes ready for production use.
Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Ron Dagostino <rdagostino@confluent.io>, Ismael Juma <ismael@juma.me.uk>, José Armando García Sancio <jsancio@apache.org>, Justine Olshan <jolshan@confluent.io>
In many parameterized tests, the display name is broken. Example - testMetadataFetch appears as [1] true, [2] false link
This is because the constant in @ParameterizedTest
String DEFAULT_DISPLAY_NAME = "[{index}] {argumentsWithNames}";
This PR adds a new junit-platform.properties which overrides to add a {displayName} which shows the the display name of the method
For existing tests which override the name, should work as is. The precedence rules are explained
name attribute in @ParameterizedTest, if present
value of the junit.jupiter.params.displayname.default configuration parameter, if present
DEFAULT_DISPLAY_NAME constant defined in @ParameterizedTest
Source: https://junit.org/junit5/docs/current/user-guide/#writing-tests-parameterized-tests-display-names
Sample test run output
Before: [1] true link
After: testMetadataExpiry(boolean).false link
This commit is an extension of bdf6d46b41 which needed to reverted due to introduces test failures.
Reviewers: David Jacot <djacot@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
This reverts commit bdf6d46b41. We found out that this commit introduced flakiness in Streams' tests. We will revise it.
Reviewers: Bruno Cadonna <cadonna@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>
When creating partition registrations directories must always be defined.
If creating a partition from a PartitionRecord or PartitionChangeRecord from an older version that
does not support directory assignments, then DirectoryId.MIGRATING is assumed.
If creating a new partition, or triggering a change in assignment, DirectoryId.UNASSIGNED should be
specified, unless the target broker has a single online directory registered, in which case the
replica should be assigned directly to that single directory.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Assign MetadataVersion.IBP_3_7_IV2 to JBOD.
Move KIP-966 support to MetadataVersion.IBP_3_7_IV3.
Create MetadataVersion.LATEST_PRODUCTION as the latest metadata version that can be used when formatting a
new cluster, or upgrading a cluster using kafka-features.sh. This will allow us to clearly distinguish between stable
and unstable metadata versions for the first time.
Reviewers: Igor Soarez <soarez@apple.com>, Ron Dagostino <rndgstn@gmail.com>, Calvin Liu <caliu@confluent.io>, Proven Provenzano <pprovenzano@confluent.io>
DirectoryId.MIGRATING should be all zeros. All zeros is the default Uuid value in KPRC, and
MIGRATING is the default directory ID value.
Reviewers: Ron Dagostino <rdagostino@confluent.io>
While any blocking operation under holding the UnifiedLog.lock could lead to serious performance (even availability) issues, currently there are several paths that calls fsync(2) inside the lock
In the meantime the lock is held, all subsequent produces against the partition may block
This easily causes all request-handlers to be busy on bad disk performance
Even worse, when a disk experiences tens of seconds of glitch (it's not rare in spinning drives), it makes the broker to unable to process any requests with unfenced from the cluster (i.e. "zombie" like status)
This PR gets rid of 4 cases of essentially-unnecessary fsync(2) calls performed under the lock:
(1) ProducerStateManager.takeSnapshot at UnifiedLog.roll
I moved fsync(2) call to the scheduler thread as part of existing "flush-log" job (before incrementing recovery point)
Since it's still ensured that the snapshot is flushed before incrementing recovery point, this change shouldn't cause any problem
(2) ProducerStateManager.removeAndMarkSnapshotForDeletion as part of log segment deletion
This method calls Utils.atomicMoveWithFallback with needFlushParentDir = true internally, which calls fsync.
I changed it to call Utils.atomicMoveWithFallback with needFlushParentDir = false (which is consistent behavior with index files deletion. index files deletion also doesn't flush parent dir)
This change shouldn't cause problems neither.
(3) LeaderEpochFileCache.truncateFromStart when incrementing log-start-offset
This path is called from deleteRecords on request-handler threads.
Here, we don't need fsync(2) either actually.
On unclean shutdown, few leader epochs might be remained in the file but it will be handled by LogLoader on start-up so not a problem
(4) LeaderEpochFileCache.truncateFromEnd as part of log truncation
Likewise, we don't need fsync(2) here, since any epochs which are untruncated on unclean shutdown will be handled on log loading procedure
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
A new AssignmentsManager accumulates, batches, and sends KIP-858
assignment events to the Controller. Assignments are sent via
AssignReplicasToDirs requests.
Move QuorumTestHarness.formatDirectories into TestUtils so it can be
used in other test contexts.
Fix a bug in ControllerRegistration.java where the wrong version of the
record was being generated in ControllerRegistration.toRecord.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>
DirectoryId.random doesn't need to instantiate the first 100 IDs to check if an ID is one of them.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Justine Olshan <jolshan@confluent.io>, Proven Provenzano <93720617+pprovenzano@users.noreply.github.com>
Reviewers: Christo Lolov <lolovc@amazon.com>, Colin P. McCabe <cmccabe@apache.org>, Proven Provenzano <pprovenzano@confluent.io>, Ron Dagostino <rdagostino@confluent.io>
Separating out the protocol changes from #14444 in an effort to more quickly unblock the client side PR.
This is the protocol changes to populate the fields in KIP-951. On NOT_LEADER_OR_FOLLOWER errors in both FETCH and PRODUCE the new leader ID and epoch are included in the response. The endpoint for the new leader is retrieved from the metadata cache. The new fields are all optional (tagged) and an IBP bump is required.
https://cwiki.apache.org/confluence/display/KAFKA/KIP-951%3A+Leader+discovery+optimisations+for+the+client
Reviewers: Justine Olshan <jolshan@confluent.io>, Mayank Shekhar Narula <mayanks.narula@gmail.com>
After a late discussion in the voting thread for KIP-858 we
decided to improve the names for the designated reserved
log directory UUID values.
Reviewers: Christo Lolov <lolovc@amazon.com>, Ismael Juma <ismael@juma.me.uk>, Ziming Deng <dengziming1993@gmail.com>.
MINOR: Server-Commons cleanup
Fixes Javadoc and minor issues in the Java files of Server-Commons modules.
Javadoc is now formatted as intended by the author of the doc itself.
Signed-off-by: Josep Prat <josep.prat@aiven.io>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
This patch introduces preliminary changes for Eligible Leader Replicas (KIP-966)
* New MetadataVersion 16 (3.7-IV1)
* New record versions for PartitionRecord and PartitionChangeRecord
* New tagged fields on PartitionRecord and PartitionChangeRecord
* New static config "eligible.leader.replicas.enable" to gate the whole feature
Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Spotbugs was temporarily disabled as part of KAFKA-15485 to support Kafka build with JDK 21. This PR upgrades the spotbugs version to 4.8.0 which adds support for JDK 21 and enables it's usage on build again.
Reviewers: Divij Vaidya <diviv@amazon.com>
This patch moves the TopicIdPartition from the metadata module to the server-common module so it can be used by the group-coordinator module as well.
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, David Jacot <djacot@confluent.io>
Add support for --bootstrap-controller in the following command-line tools:
- kafka-cluster.sh
- kafka-configs.sh
- kafka-features.sh
- kafka-metadata-quorum.sh
To implement this, the following AdminClient APIs now support the new bootstrap.controllers
configuration:
- Admin.alterConfigs
- Admin.describeCluster
- Admin.describeConfigs
- Admin.describeFeatures
- Admin.describeMetadataQuorum
- Admin.incrementalAlterConfigs
- Admin.updateFeatures
Command-line tool changes:
- Add CommandLineUtils.initializeBootstrapProperties to handle parsing --bootstrap-controller
in addition to --bootstrap-server.
- Add --bootstrap-controller to ConfigCommand.scala, ClusterTool.java, FeatureCommand.java, and
MetadataQuorumCommand.java.
KafkaAdminClient changes:
- Add the AdminBootstrapAddresses class to handle extracting bootstrap.servers or
bootstrap.controllers from the config map for KafkaAdminClient.
- In AdminMetadataManager, store the new usingBootstrapControllers boolean. Generalize
authException to encompass the concept of fatal exceptions in general. (For example, the
fatal exception where we talked to the wrong node type.) Treat
MismatchedEndpointTypeException and UnsupportedEndpointTypeException as fatal exceptions.
- Extend NodeProvider to include information about whether bootstrap.controllers is supported.
- Modify the APIs described above to support bootstrap.controllers.
Server-side changes:
- Support DescribeConfigsRequest on kcontrollers.
- Add KRaftMetadataCache to the kcontroller to simplify implemeting describeConfigs (and
probably more APIs in the future). It's mainly a wrapper around MetadataImage, so there is
essentially no extra resource consumption.
- Split RuntimeLoggerManager out of ConfigAdminManager to handle the incrementalAlterConfigs
support for BROKER_LOGGER. This is now supported on kcontrollers as well as brokers.
- Fix bug in AuthHelper.computeDescribeClusterResponse that resulted in us always sending back
BROKER as the endpoint type, even on the kcontroller.
Miscellaneous:
- Fix a few places in exceptions and log messages where we wrote "broker" instead of "node".
For example, an exception in NodeApiVersions.java, and a log message in NetworkClient.java.
- Fix the slf4j log prefix used by KafkaRequestHandler logging so that request handlers on a
controller don't look like they're on a broker.
- Make the FinalizedVersionRange constructor public for the sake of a junit test.
- Add unit and integration tests for the above.
Reviewers: David Arthur <mumrah@gmail.com>, Doguscan Namal <namal.doguscan@gmail.com>
This is one of the steps required for kafka to compile with Java 21.
For each case, one of the following fixes were applied:
1. Suppress warning if fixing would potentially result in an incompatible change (for public classes)
2. Add final to one or more methods so that the escape is not possible
3. Replace method calls with direct field access.
In addition, we also fix a couple of compiler warnings related to deprecated references in the `core` module.
See the following for more details regarding the new lint warning:
https://www.oracle.com/java/technologies/javase/21-relnote-issues.html#JDK-8015831
Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Chris Egerton <chrise@aiven.io>
Implement KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add
Controller Registration. This KIP adds a new version of DescribeClusterRequest which is supported
by KRaft controllers. It also teaches AdminClient how to use this new DESCRIBE_CLUSTER request to
talk directly with the controller quorum. This is all gated behind a new MetadataVersion,
IBP_3_7_IV0.
In order to share the DESCRIBE_CLUSTER logic between broker and controller, this PR factors it out
into AuthHelper.computeDescribeClusterResponse.
The KIP adds three new errors codes: MISMATCHED_ENDPOINT_TYPE, UNSUPPORTED_ENDPOINT_TYPE, and
UNKNOWN_CONTROLLER_ID. The endpoint type errors can be returned from DescribeClusterRequest
On the controller side, the controllers now try to register themselves with the current active
controller, by sending a CONTROLLER_REGISTRATION request. This, in turn, is converted into a
RegisterControllerRecord by the active controller. ClusterImage, ClusterDelta, and all other
associated classes have been upgraded to propagate the new metadata. In the metadata shell, the
cluster directory now contains both broker and controller subdirectories.
QuorumFeatures previously had a reference to the ApiVersions structure used by the controller's
NetworkClient. Because this PR removes that reference, QuorumFeatures now contains only immutable
data. Specifically, it contains the current node ID, the locally supported features, and the list
of quorum node IDs in the cluster.
Reviewers: David Arthur <mumrah@gmail.com>, Ziming Deng <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
This implementation introduces two new configurations `log.message.timestamp.before.max.ms` and `log.message.timestamp.after.max.ms` and deprecates `log.message.timestamp.difference.max.ms`.
The default value for all these three configs is maintained to be Long.MAX_VALUE for backward compatibility but with the newly added configurations we can have a finer control when validating message timestamps that are in the past and the future compared to the broker's timestamp.
To maintain backward compatibility if the default value of `log.message.timestamp.before.max.ms` is not changed, we are assuming users are still using the deprecated config `log.message.timestamp.difference.max.ms` and validation is done using its value. This ensures that existing customers who have customized the value of `log.message.timestamp.difference.max.ms` will continue to see no change in behavior.
Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
#14083 added support for delegation tokens in KRaft and attached that support to the existing
MetadataVersion 3.6-IV1. This patch moves that support into a separate MetadataVersion 3.6-IV2.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This PR contains three main changes:
- Support for transactions in MetadataLoader
- Abort in-progress transaction during controller failover
- Utilize transactions for ZK to KRaft migration
A new MetadataBatchLoader class is added to decouple the loading of record batches from the
publishing of metadata in MetadataLoader. Since a transaction can span across multiple batches (or
multiple transactions could exist within one batch), some buffering of metadata updates was needed
before publishing out to the MetadataPublishers. MetadataBatchLoader accumulates changes into a
MetadataDelta, and uses a callback to publish to the publishers when needed.
One small oddity with this approach is that since we can "splitting" batches in some cases, the
number of bytes returned in the LogDeltaManifest has new semantics. The number of bytes included in
a batch is now only included in the last metadata update that is published as a result of a batch.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Viktor Somogyi <viktor.somogyi@cloudera.com>
Implement the QuorumController side of KRaft metadata transactions.
As specified in KIP-868, this PR creates a new metadata version, IBP_3_6_IV1, which contains the
three new records: AbortTransactionRecord, BeginTransactionRecord, EndTransactionRecord.
In order to make offset management unit-testable, this PR moves it out of QuorumController.java and
into OffsetControlManager.java. The general approach here is to track the "last stable offset," which is
calculated by looking at the latest committed offset and the in-progress transaction (if any). When
a transaction is aborted, we revert back to this last stable offset. We also revert back to it when
the controller is transitioning from active to inactive.
In a follow-up PR, we will add support for the transaction records in MetadataLoader. We will also
add support for automatically aborting pending transactions after a controller failover.
Reviewers: David Arthur <mumrah@gmail.com>
A HistoricalIterator at epoch N is supposed to only reveal elements at epoch N or earlier. However,
due to a bug, we sometimes will reveal elements which are at a newer epoch than N. The bug does
not affect elements that are in the latest epoch (aka topTier). It only affects elements that are
newer than N, but which do not persist until the latest epoch. This PR fixes the bug and adds a
unit test for this case.
Reviewers: David Arthur <mumrah@gmail.com>
This patch does a few things:
1) It introduces the `OffsetAndMetadata` class which hold the committed offsets in the group coordinator.
2) It adds methods to deal with OffsetCommit records to `RecordHelpers`.
3) It adds `MetadataVersion#offsetCommitValueVersion` to get the version of the OffsetCommit value record that should be used.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Arthur <mumrah@gmail.com>, Justine Olshan <jolshan@confluent.io>
Introduced extra mapping to track verification state.
When verifying, there is a race condition that the add partitions verification response returns that the partition is in the ongoing transaction, but an abort marker is written before we get to append. Therefore, we track any given transaction we are verifying with an object unique to that transaction.
We check this unique state upon the first append to the log. After that, we can rely on currentTransactionFirstOffset. We remove the verification state on appending to the log with a transactional data record or marker.
We will also clean up lingering verification state entries via the producer state entry expiration mechanism. We do not update the the timestamp on retrying a verification for a transaction, so each entry must be verified before producer.id.expiration.ms.
There were a few other fixes:
- Moved the transaction manager handling for failed batch into the future completed exceptionally block to avoid processing it twice (this caused issues in unit tests)
- handle interrupted exceptions encountered when callback thread encountered them
- change handling to throw error if we try to set verification state and leaderLogIfLocal is None.
Reviewers: David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>
This patch rewrites MockTimer in Java and moves it from core to server-common. This continues the work started in https://github.com/apache/kafka/pull/13820.
Reviewers: Divij Vaidya <diviv@amazon.com>
This fixes a regression introduced by the previous KAFKA-15109 commit (d0457f7360 on trunk).
Reviewers: Colin P. McCabe <cmccabe@apache.org>, José Armando García Sancio <jsancio@apache.org>
RPCProducerIdManager initiates an async request to the controller to grab a block of producer IDs and then blocks waiting for a response from the controller.
This is done in the request handler threads while holding a global lock. This means that if many producers are requesting producer IDs and the controller is slow to respond, many threads can get stuck waiting for the lock.
This patch aims to:
* resolve the deadlock scenario mentioned above by not waiting for a new block and returning an error immediately
* remove synchronization usages in RpcProducerIdManager.generateProducerId()
* handle errors returned from generateProducerId() so that KafkaApis does not log unexpected errors
* confirm producers backoff before retrying
* introduce backoff if manager fails to process AllocateProducerIdsResponse
Reviewers: Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>
Use thread safe Caffeine to cache indexes fetched from RemoteTier locally. This PR removes a lock contention that led to higher fetch latencies as the IO threads spent time unnecessarily waiting on global cache lock while a single thread fetches the index from remote tier. See PR #13850 for details and rejected alternatives.
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
The Java rewrite is kept relatively close to the Scala original
to minimize potential newly introduced bugs and to make reviewing
simpler. The following details might be of note:
- The `Logging` trait moved to InterBrokerSendThread with the
rewrite of ShutdownableThread has been similarly moved to any
subclasses that currently use it. InterBrokerSendThread's own
logging has been made to use ShutdownableThread's logger which
mimics the prefix/log identifier that the trait provided.
- The case RequestAndCompletionHandler class has been made a
separate POJO class and the internal-use UnsentRequests class
has been kept as a static nested class.
- The relatively commonly used but internal (not part of the
public API) clients classes that InterBrokerSendThread relies on
have been allowlisted in the server-common import control.
- The accompanying test class has also been moved and rewritten
with one new test added and most of the pre-existing tests made
stricter.
Reviewers: David Jacot <djacot@confluent.io>
This PR expands the scope of ApiVersionManager a bit to include returning the current
MetadataVersion and features that are in effect. This is useful in general because that information
needs to be returned in an ApiVersionsResponse. It also allows us to fix the ApiVersionManager
interface so that all subclasses implement all methods of the interface. Having subclasses that
don't implement some methods is dangerous because they could cause exceptions at runtime in
unexpected scenarios.
On the KRaft controller, we were previously performing a read operation in the QuorumController
thread to get the current metadata version and features. With this PR, we now read a volatile
variable maintained by a separate MetadataVersionContextPublisher object. This will improve
performance and simplify the code. It should not change the guarantees we are providing; in both
the old and new scenarios, we need to be robust against version skew scenarios during updates.
Add a Features class which just has a 3-tuple of metadata version, features, and feature epoch.
Remove MetadataCache.FinalizedFeaturesAndEpoch, since it just duplicates the Features class.
(There are some additional feature-related classes that can be consolidated in in a follow-on PR.)
Create a java class, EndpointReadyFutures, for managing the futures associated with individual
authorizer endpoints. This avoids code duplication between ControllerServer and BrokerServer and
makes this code unit-testable.
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
Change Timer.java to implement AutoCloseable because automatic bug finders will flag a warning if an object of a class is marked as AutoCloseable but is not closed properly in the code.
Reviewers: Divij Vaidya <diviv@amazon.com>
This patch rewrite `Timer` and the related classes in Java and moves them to `server-common` module. It is basically a one to one rewrite of the Scala code. Note that `MockTimer` is not moved as part of this patch. It will be done separately.
Reviewers: Divij Vaidya <diviv@amazon.com>
This patch rewrite `MockTime` in Java and moves it to `server-common` module. This is a prerequisite to move `MockTimer` later on to `server-common` as well.
Reviewers: David Arthur <mumrah@gmail.com>
When the KRaft controller removes a replica from the ISR because of the controlled shutdown there is no need for the leader epoch to be increased by the KRaft controller. This is accurate as long as the topic partition leader doesn't add the removed replica back to the ISR.
This change also fixes a bug when computing the HWM. When computing the HWM, replicas that are not eligible to join the ISR but are caught up should not be included in the computation. Otherwise, the HWM will never increase for replica.lag.time.max.ms because the shutting down replica is not sending FETCH request. Without this additional fix PRODUCE requests would timeout if the request timeout is greater than replica.lag.time.max.ms.
Because of the bug above the KRaft controller needs to check the MV to guarantee that all brokers support this bug fix before skipping the leader epoch bump.
Reviewers: David Mao <47232755+splett2@users.noreply.github.com>, Divij Vaidya <diviv@amazon.com>, David Jacot <djacot@confluent.io>
Previously, if a user tried to perform an overly large batch operation on the KRaft controller
(such as creating a million topics), we would create a very large number of records in memory. Our
attempt to write these records to the Raft layer would fail, because there were too many to fit in
an atomic batch. This failure, in turn, would trigger a controller failover.
(Note: I am assuming here that no topic creation policy was in place that would prevent the
creation of a million topics. I am also assuming that the user operation must be done atomically,
which is true for all current user operations, since we have not implemented KIP-868 yet.)
With this PR, we fail immediately when the number of records we have generated exceeds the
threshold that we can apply. This failure does not generate a controller failover. We also now
fail with a PolicyViolationException rather than an UnknownServerException.
In order to implement this in a simple way, this PR adds the BoundedList class, which wraps any
list and adds a maximum length. Attempts to grow the list beyond this length cause an exception to
be thrown.
Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ijuma@apache.org>, Divij Vaidya <diviv@amazon.com>
When creating the QuorumController, log whether ZK migration is enabled.
When applying a feature level record which sets the metadata version, log the metadata version enum
rather than the numeric feature level.
Improve the logging when we replay snapshots in QuorumController. Log both the beginning and the
end of replay.
When TRACE is enabled, log every record that is replayed in QuorumController. Since some records
may contain sensitive information, create RecordRedactor to assist in logging only what is safe to
put in the log4j file.
Add logging to ControllerPurgatory. Successful completions are logged at DEBUG; failures are logged
at INFO, and additions are logged at TRACE.
Remove SnapshotReason.java, SnapshotReasonTest.java, and
QuorumController#generateSnapshotScheduled. They are deadcode now that snapshot generation moved to
org.apache.kafka.image.publisher.SnapshotGenerator.
Reviewers: David Arthur <mumrah@gmail.com>, José Armando García Sancio <jsancio@apache.org>
Fix NPE while merging the deltatable. Because it's possible that hashTier is
not null but deltatable is null (ex: removing data), we should have null check
while merging for deltatable like other places did. Also added tests that will
fail without this change.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Saw this error message in log:
ERROR [QuorumController id=1] writeNoOpRecord: unable to start processing because of RejectedExecutionException. Reason: null (org.apache.kafka.controller.QuorumController)
The null reason is not helpful with only RejectedExecutionException. Adding the reason to it.
Reviewers: David Arthur <mumrah@gmail.com>, Divij Vaidya <diviv@amazon.com>, Manyanda Chitimbo <manyanda.chitimbo@gmail.com>
This patch renames from `ControllerPurgatory` to `DeferredEventQueue` and moves it from the `metadata` module to `server-common` module.
Reviewers: Alexandre Dupriez <alexandre.dupriez@gmail.com>, Ziming Deng <dengziming1993@gmail.com>, José Armando García Sancio <jsancio@apache.org>
Currently, StandardAuthorizer uses a R/W lock for maintaining the consistency of data. For the clusters with very high traffic, we will typically see an increase in latencies whenever a write operation comes. The intent of this PR is to get rid of the R/W lock with the help of immutable or persistent collections. Basically, new object references are used to hold the intermediate state of the write operation. After the completion of the operation, the main reference to the cache is changed to point to the new object. Also, for the read operation, the code is changed such that all accesses to the cache for a single read operation are done to a particular cache object only.
In the PR description, you can find the performance of various libraries at the time of both read and write. Read performance is checked with the existing AuthorizerBenchmark. For write performance, a new AuthorizerUpdateBenchmark has been added which evaluates the performance of the addAcl operation.
Reviewers: Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Divij Vaidya <diviv@amazon.com>
Rework UserScramCredentialRecord to store serverKey and StoredKey rather than saltedPassword. This
is necessary to support migration from ZK, since those are the fields we stored in ZK. Update
latest MetadataVersion to IBP_3_5_IV2 and make SCRAM support conditional on this version. Moved
ScramCredentialData.java from org.apache.kafka.image to org.apache.kafka.metadata, which seems more
appropriate.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
topic counts.
Introduces the use of persistent data structures in the KRaft metadata image to avoid copying the entire TopicsImage upon every change. Performance that was O(<number of topics in the cluster>) is now O(<number of topics changing>), which has dramatic time and GC improvements for the most common topic-related metadata events. We abstract away the chosen underlying persistent collection library via ImmutableMap<> and ImmutableSet<> interfaces and static factory methods.
Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>, Purshotam Chauhan <pchauhan@confluent.io>
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>