This patch fixes two issues with IncrementalAlterConfigs and the ZK migration. First, it changes the handling of IncrementalAlterConfigs to check if the controller is ZK vs KRaft and only forward for KRaft. Second, it adds a check in KafkaZkClient#setOrCreateEntityConfigs to ensure a ZK broker is not directly modifying configs in ZK if there is a KRaft controller. This closes the race condition between KRaft taking over as the active controller and the ZK brokers learning about this.
*Forwarding*
During the ZK migration, there is a time when the ZK brokers are running with migrations enabled, but KRaft has yet to take over as the controller. Prior to KRaft taking over as the controller, the ZK brokers in migration mode were unconditionally forwarding IncrementalAlterConfigs (IAC) to the ZK controller. This works for some config types, but breaks when setting BROKER and BROKER_LOGGER configs for a specific broker. The behavior in KafkaApis for IAC was to always forward if the forwarding manager was defined. Since ZK brokers in migration mode have forwarding enabled, the forwarding would happen, and the special logic for BROKER and BROKER_LOGGER would be missed, causing the request to fail.
With this fix, the IAC handler will check if the controller is KRaft or ZK and only forward for KRaft.
*Protected ZK Writes*
As part of KIP-500, we moved most (but not all) ZK mutations to the ZK controller. One of the things we did not move fully to the controller was entity configs. This is because there was some special logic that needed to run on the broker for certain config updates. If a broker-specific config was set, AdminClient would route the request to the proper broker. In KRaft, we have a different mechanism for handling broker-specific config updates.
Leaving this ZK update on the broker side would be okay if we were guarding writes on the controller epoch, but it turns out KafkaZkClient#setOrCreateEntityConfigs does unprotected "last writer wins" updates to ZK. This means a ZK broker could update the contents of ZK after the metadata had been migrated to KRaft. No good! To fix this, this patch adds a check on the controller epoch to KafkaZkClient#setOrCreateEntityConfigs but also adds logic to fail the update if the controller is a KRaft controller.
The new logic in setOrCreateEntityConfigs adds STALE_CONTROLLER_EPOCH as a new exception that can be thrown while updating configs.
Reviewers: Luke Chen <showuon@gmail.com>, Akhilesh Chaganti <akhileshchg@users.noreply.github.com>, Chia-Ping Tsai <chia7712@gmail.com>
This patch add three benchmarks for the client assignors, the server assignors and the target assignment builder.
Reviewers: David Jacot <djacot@confluent.io>
This pr fixes the bug created by #15263 which caused topic partition to be recreated whenever the original log dir is offline: Log directory failure re-creates partitions in another logdir automatically
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>, Gaurav Narula <gaurav_narula2@apple.com>, Proven Provenzano <pprovenzano@confluent.io>
1) This PR moves kafka.security classes from core to server module.
2) AclAuthorizer not moved, because it has heavy dependencies on core classes that not rewrited from scala at the moment.
3) AclAuthorizer will be deleted as part of ZK removal
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
Co-authored-by: n.izhikov <n.izhikov@vk.team>
Expiration of ProducerIds is implemented with a slow removal of map keys:
producers.keySet().removeAll(keys);
Unnecessarily going through all producer ids and then throw all expired keys to be removed.
This leads to exponential time on worst case when most/all keys need to be removed:
Benchmark (numProducerIds) Mode Cnt Score Error Units
ProducerStateManagerBench.testDeleteExpiringIds 100 avgt 3 9164.043 ± 10647.877 ns/op
ProducerStateManagerBench.testDeleteExpiringIds 1000 avgt 3 341561.093 ± 20283.211 ns/op
ProducerStateManagerBench.testDeleteExpiringIds 10000 avgt 3 44957983.550 ± 9389011.290 ns/op
ProducerStateManagerBench.testDeleteExpiringIds 100000 avgt 3 5683374164.167 ± 1446242131.466 ns/op
A simple fix is to use map#remove(key) instead, leading to a more linear growth:
Benchmark (numProducerIds) Mode Cnt Score Error Units
ProducerStateManagerBench.testDeleteExpiringIds 100 avgt 3 5779.056 ± 651.389 ns/op
ProducerStateManagerBench.testDeleteExpiringIds 1000 avgt 3 61430.530 ± 21875.644 ns/op
ProducerStateManagerBench.testDeleteExpiringIds 10000 avgt 3 643887.031 ± 600475.302 ns/op
ProducerStateManagerBench.testDeleteExpiringIds 100000 avgt 3 7741689.539 ± 3218317.079 ns/op
Flamegraph of the CPU usage at dealing with expiration when producers ids ~1Million:
Reviewers: Justine Olshan <jolshan@confluent.io>
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>
Improve JsonConverter performance by using afterBurnModule of Jackson library.
Reviewers: Divij Vaidya <diviv@amazon.com>, Mickael Maison <mickael.maison@gmail.com>
This patch adds the concept of a "Full" UpdateMetadataRequest, similar to what is used in
LeaderAndIsr. A new tagged field is added to UpdateMetadataRequest at version 8 which allows the
KRaft controller to indicate if a UMR contains all the metadata or not. Since UMR is implicitly
treated as incremental by the ZK broker, we needed a way to detect topic deletions when the KRaft
broker sends a metadata snapshot to the ZK broker. By sending a "Full" flag, the broker can now
compare existing topic IDs to incoming topic IDs and calculate which topics should be removed from
the MetadataCache.
This patch only removes deleted topics from the MetadataCache. Partition/log management was
implemented in KAFKA-15605.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
The PR includes:
* Added a new class of CleanShutdownFile which helps write and read from a clean shutdown file.
* Updated the BrokerRegistration API.
* Client side handling for the broker epoch.
* Minimum work on the controller side.
Reviewers: Jun Rao <junrao@gmail.com>
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>
Only initialize remote topic metrics when system-wise remote storage is enabled to avoid impacting performance for existing brokers. Also add tests.
Reviewers: Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
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 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>
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>
After this change,
For broker side decompression: JMH benchmark RecordBatchIterationBenchmark demonstrates 20-70% improvement in throughput (see results for RecordBatchIterationBenchmark.measureSkipIteratorForVariableBatchSize).
For consumer side decompression: JMH benchmark RecordBatchIterationBenchmark a mix bag of single digit regression for some compression type to 10-50% improvement for Zstd (see results for RecordBatchIterationBenchmark.measureStreamingIteratorForVariableBatchSize).
Reviewers: Luke Chen <showuon@gmail.com>, Manyanda Chitimbo <manyanda.chitimbo@gmail.com>, Ismael Juma <mail@ismaeljuma.com>
I have moved this config into producer state manager so it can be checked easily under the log lock when we are about to append.
Only a few test files currently use the validation and those have been verified to work via running the tests.
Reviews: David Jacot <djacot@confluent.io>
Motivation
Reading/writing the protocol buffer varInt32 and varInt64 (also called varLong in our code base) is in the hot path of data plane code in Apache Kafka. We read multiple varInt in a record and in long. Hence, even a minor change in performance could extrapolate to larger performance benefit.
In this PR, we only update varInt32 encoding/decoding.
Changes
This change uses loop unrolling and reduces the amount of repetition of calculations. Based on the empirical results from the benchmark, the code has been modified to pick up the best implementation.
Results
Performance has been evaluated using JMH benchmarks on JDK 17.0.6. Various implementations have been added in the benchmark and benchmarking has been done for different sizes of varints and varlongs. The benchmark for various implementations have been added at ByteUtilsBenchmark.java
Reviewers: Ismael Juma <mlists@juma.me.uk>, Luke Chen <showuon@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>
1. add ZkMigrationReady in apiVersionsResponse
2. check all nodes if ZkMigrationReady are ready before moving to next migration state
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>
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>
We are handling complex workflows ListOffsets by chaining together MetadataCall instances and ListOffsetsCall instances, there are many complex and error-prone logic. In this PR we rewrote it with the `AdminApiDriver` infra, notable changes better than old logic:
1. Retry lookup stage on receiving `NOT_LEADER_OR_FOLLOWER` and `LEADER_NOT_AVAILABLE`, whereas in the past we failed the partition directly without retry.
2. Removing class field `supportsMaxTimestamp` and calculating it on the fly to avoid the mutable state, this won't change any behavior of the client.
3. Retry fulfillment stage on `RetriableException`, whereas in the past we just retry fulfillment stage on `InvalidMetadataException`, this means we will retry on `TimeoutException` and other `RetriableException`.
We also `handleUnsupportedVersionException` to `AdminApiHandler` and `AdminApiLookupStrategy`, they are used to keep consistency with old logic, and we can continue improvise them.
Reviewers: Ziming Deng <dengziming1993@gmail.com>, David Jacot <djacot@confluent.io>
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>
As the third part of the KIP-903, it fills the broker epochs from the Fetch request into the AlterPartitionRequest. Also, before generating the alterPartitionRequest, the partition will check whether the broker epoch from the FetchRequest matches with the broker epoch recorded in the metadata cache. If not, the ISR change will be delayed.
Reviewers: Jun Rao <junrao@gmail.com>
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 patch does a few things:
1) It introduces a new flag to the request spec: `latestVersionUnstable`. It signifies that the last version of the API is considered unstable (or still in development). As such, the last API version is not exposed by the server unless specified otherwise with the new internal `unstable.api.versions.enable`. This allows us to commit new APIs which are still in development.
3) It adds the ConsumerGroupHeartbeat API, part of KIP-848, and marks it as unreleased for now.
4) It adds the new error codes required by the new ConsumerGroupHeartbeat API.
Reviewers: Justine Olshan <jolshan@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Jason Gustafson <jason@confluent.io>
This patch migrates all the internal APIs of the current group coordinator to the new `GroupCoordinator` interface. It also makes the current implementation package private to ensure that it is not used anymore.
Reviewers: Justine Olshan <jolshan@confluent.io>
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>
This patch introduces a preliminary state machine that can be used by KRaft
controller to drive online migration from Zk to KRaft.
MigrationState -- Defines the states we can have while migration from Zk to
KRaft.
KRaftMigrationDriver -- Defines the state transitions, and events to handle
actions like controller change, metadata change, broker change and have
interfaces through which it claims Zk controllership, performs zk writes and
sends RPCs to ZkBrokers.
MigrationClient -- Interface that defines the functions used to claim and
relinquish Zk controllership, read to and write from Zk.
Co-authored-by: David Arthur <mumrah@gmail.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
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>
For broader context on this change, please check:
* KAFKA-14470: Move log layer to storage module
Reviewers: dengziming <dengziming1993@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Satish Duggana <satishd@apache.org>, Ismael Juma <ismael@juma.me.uk>
Also improved `LogValidatorTest` to cover a bug that was originally
only caught by `LogAppendTimeTest`.
For broader context on this change, please check:
* KAFKA-14470: Move log layer to storage module
Reviewers: Jun Rao <junrao@gmail.com>
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>
This PR implements the follower fetch protocol as mentioned in KIP-405.
Added a new version for ListOffsets protocol to receive local log start offset on the leader replica. This is used by follower replicas to find the local log star offset on the leader.
Added a new version for FetchRequest protocol to receive OffsetMovedToTieredStorageException error. This is part of the enhanced fetch protocol as described in KIP-405.
We introduced a new field locaLogStartOffset to maintain the log start offset in the local logs. Existing logStartOffset will continue to be the log start offset of the effective log that includes the segments in remote storage.
When a follower receives OffsetMovedToTieredStorage, then it tries to build the required state from the leader and remote storage so that it can be ready to move to fetch state.
Introduced RemoteLogManager which is responsible for
initializing RemoteStorageManager and RemoteLogMetadataManager instances.
receives any leader and follower replica events and partition stop events and act on them
also provides APIs to fetch indexes, metadata about remote log segments.
Followup PRs will add more functionality like copying segments to tiered storage, retention checks to clean local and remote log segments. This will change the local log start offset and make sure the follower fetch protocol works fine for several cases.
You can look at the detailed protocol changes in KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication
Co-authors: satishd@apache.org, kamal.chandraprakash@gmail.com, yingz@uber.com
Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Cong Ding <cong@ccding.com>, Tirtha Chatterjee <tirtha.p.chatterjee@gmail.com>, Yaodong Yang <yangyaodong88@gmail.com>, Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Jun Rao <junrao@gmail.com>
This PR 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>
When a consumer makes a fetch request to a follower (KIP-392), the fetch request will sit in the purgatory until `fetch.max.wait.ms` is reached because the purgatory is not completed after replication. This patch aims to complete the delayed fetch purgatory after successfully replicating from the leader.
Reviewers: Artem Livshits <alivshits@confluent.io>, Luke Chen <showuon@gmail.com>, David Jacot <djacot@confluent.io>
Implementation for KIP-831.
1. add remainingLogsToRecover metric for the number of remaining logs for each log.dir to be recovered
2. add remainingSegmentsToRecover metric for the number of remaining segments for the current log assigned to the recovery thread.
3. remove these metrics after log loaded completely
4. add tests
Reviewers: Jun Rao <jun@confluent.io>, Tom Bentley <tbentley@redhat.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 refactors the leader API access in the follower fetch path.
Added a LeaderEndPoint interface which serves all access to the leader.
Added a LocalLeaderEndPoint and a RemoteLeaderEndPoint which implements the LeaderEndPoint interface to handle fetches from leader in local & remote storage respectively.
Reviewers: David Jacot <djacot@confluent.io>, Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
When a partition leader receives a `Fetch` request from a replica which is not in the current replica set, the behavior today is to return a successful fetch response, but with empty data. This causes the follower to retry until metadata converges without updating any state on the leader side. It is clearer in this case to return an error, so that the metadata inconsistency is visible in logging and so that the follower backs off before retrying.
In this patch, we use `UNKNOWN_LEADER_EPOCH` when the `Fetch` request includes the current leader epoch. The way we see this is that the leader is validating the (replicaId, leaderEpoch) tuple. When the leader returns `UNKNOWN_LEADER_EPOCH`, it means that the leader does not expect the given leaderEpoch from that replica. If the request does not include a leader epoch, then we use `NOT_LEADER_OR_FOLLOWER`. We can take a similar interpretation for this case: the leader is rejecting the request because it does not think it should be the leader for that replica. But mainly these errors ensure that the follower will retry the request.
As a part of this patch, I have refactored the way that the leader updates follower fetch state. Previously, the process is a little convoluted. We send the fetch from `ReplicaManager` down to `Partition.readRecords`, then we iterate over the results and call `Partition.updateFollowerFetchState`. It is more straightforward to update state directly as a part of `readRecords`. All we need to do is pass through the `FetchParams`. This also prevents an unnecessary copy of the read results.
Reviewers: David Jacot <djacot@confluent.io>
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>
Since we have changed the `AlterIsr` API to `AlterPartition`, it makes sense to rename `AlterIsrManager` as well and some of the associated classes.
Reviewers: dengziming <dengziming1993@gmail.com>, David Jacot <djacot@confluent.io>
This patch does some initial cleanups in the context of KAFKA-13790. Mainly, it renames `ZkVersion` field to `PartitionEpoch` in the `LeaderAndIsrRequest`, the `LeaderAndIsr` and the `Partition`.
Reviewers: Jason Gustafson <jason@confluent.io>, dengziming <dengziming1993@gmail.com>
jmh.sh runs tasks in quiet mode which swallows compiler errors. This is a pain and I frequently have to edit the shell script to see the error.
Reviewers: Ismael Juma <ismael@confluent.io>, Bill Bejeck <bbejeck@apache.org>
With the changes for topic IDs, we have a different flow. When a broker receives a request, it uses a map to convert the topic ID to topic names. If the topic ID is not found in the map, we return a top level error and close the session. This decision was motivated by the difficulty to store “unresolved” partitions in the session. In earlier iterations we stored an “unresolved” partition object in the cache, but it was somewhat hard to reason about and required extra logic to try to resolve the topic ID on each incremental request and add to the session. It also required extra logic to forget the topic (either by topic ID if the topic name was never known or by topic name if it was finally resolved when we wanted to remove from the session.)
One helpful simplifying factor is that we only allow one type of request (uses topic ID or does not use topic ID) in the session. That means we can rely on a session continuing to have the same information. We don’t have to worry about converting topics only known by name to topic ID for a response and we won’t need to convert topics only known by ID to name for a response.
This PR introduces a change to store the "unresolved partitions" in the cached partition object. If a version 13+ request is sent with a topic ID that is unknown, a cached partition will be created with that fetch request data and a null topic name. On subsequent incremental requests, unresolved partitions may be resolved with the new IDs found in the metadata cache. When handling the request, getting all partitions will return a TopicIdPartition object that will be used to handle the request and build the response. Since we can rely on only one type of request (with IDs or without), the cached partitions map will have different keys depending on what fetch request version is being used.
This PR involves changes both in FetchSessionHandler and FetchSession. Some major changes are outlined below.
1. FetchSessionHandler: Forgetting a topic and adding a new topic with the same name - We may have a case where there is a topic foo with ID 1 in the session. Upon a subsequent metadata update, we may have topic foo with ID 2. This means that topic foo has been deleted and recreated. When sending fetch requests version 13+ we will send a request to add foo ID 2 to the session and remove foo ID 1. Otherwise, we will fall back to the same behavior for versions 12 and below
2. FetchSession: Resolving in Incremental Sessions - Incremental sessions contain two distinct sets of partitions. Partitions that are sent in the latest request that are new/updates/forgotten partitions and the partitions already in the session. If we want to resolve unknown topic IDs we will need to handle both cases.
* Partitions in the request - These partitions are either new or updating/forgetting previous partitions in the session. The new partitions are trivial. We either have a resolved partition or create a partition that is unresolved. For the other cases, we need to be a bit more careful.
* For updated partitions we have a few cases – keep in mind, we may not programmatically know if a partition is an update:
1. partition in session is resolved, update is resolved: trivial
2. partition in session is unresolved, update is unresolved: in code, this is equivalent to the case above, so trivial as well
3. partition in session is unresolved, update is resolved: this means the partition in the session does not have a name, but the metadata cache now contains the name – to fix this we can check if there exists a cached partition with the given ID and update it both with the partition update and with the topic name.
4. partition in session is resolved, update is unresolved: this means the partition in the session has a name, but the update was unable to be resolved (ie, the topic is deleted) – this is the odd case. We will look up the partition using the ID. We will find the old version with a name but will not replace the name. This will lead to an UNKNOWN_TOPIC_OR_PARTITION or INCONSISTENT_TOPIC_ID error which will be handled with a metadata update. Likely a future request will forget the partition, and we will be able to do so by ID.
5. Two partitions in the session have IDs, but they are different: only one topic ID should exist in the metadata at a time, so likely only one topic ID is in the fetch set. The other one should be in the toForget. We will be able to remove this partition from the session. If for some reason, we don't try to forget this partition — one of the partitions in the session will cause an inconsistent topic ID error and the metadata for this partition will be refreshed — this should result in the old ID being removed from the session. This should not happen if the FetchSessionHandler is correctly in sync.
* For the forgotten partitions we have the same cases:
1. partition in session is resolved, forgotten is resolved: trivial
2. partition in session is unresolved, forgotten is unresolved: in code, this is equivalent to the case above, so trivial as well
3. partition in session is unresolved, forgotten is resolved: this means the partition in the session does not have a name, but the metadata cache now contains the name – to fix this we can check if there exists a cached partition with the given ID and try to forget it before we check the resolved name case.
4. partition in session is resolved, update is unresolved: this means the partition in the session has a name, but the update was unable to be resolved (ie, the topic is deleted) We will look up the partition using the ID. We will find the old version with a name and be able to delete it.
5. both partitions in the session have IDs, but they are different: This should be the same case as described above. If we somehow do not have the ID in the session, no partition will be removed. This should not happen unless the Fetch Session Handler is out of sync.
* Partitions in the session - there may be some partitions in the session already that are unresolved. We can resolve them in forEachPartition using a method that checks if the partition is unresolved and tries to resolve it using a topicName map from the request. The partition will be resolved before the function using the cached partition is applied.
Reviewers: David Jacot <djacot@confluent.io>
Put ZkMetadataCache in the kafka.server.metadata package rather than the kafka.server package, so
that its package is consistent with its position in the source directory hierarchy.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Before we used the metadata cache to determine whether or not to use topic IDs. Unfortunately, metadata cache updates with ZK controllers are in a separate request and may be too slow for the fetcher thread. This results in switching between topic names and topic IDs for topics that could just use IDs.
This patch adds topic IDs to FetcherState created in LeaderAndIsr requests. It also supports updating this state for follower threads as soon as a LeaderAndIsr request provides a topic ID.
We've opted to only update replica fetcher threads. AlterLogDir threads will use either topic name or topic ID depending on what was present when they were created.
Reviewers: David Jacot <djacot@confluent.io>
The ReplicaManager, LogManager, and KafkaApis class all have many
constructor parameters. It is often difficult to add or remove a
parameter, since there are so many locations that need to be updated. In
order to address this problem, we should use named parameters when
constructing these objects from Scala code. This will make it easy to
add new optional parameters without modifying many test cases. It will
also make it easier to read git diffs and PRs, since the parameters will
have names next to them. Since Java does not support named paramters,
this PR adds several Builder classes which can be used to achieve the
same effect.
ReplicaManager also had a secondary constructor, which this PR removes.
The function of the secondary constructor was just to provide some
default parameters for the main constructor. However, it is simpler just
to actually use default parameters.
Reviewers: David Arthur <mumrah@gmail.com>
The leader assumes that there is always an in-memory snapshot at the last
committed offset. This means that the controller needs to generate an in-memory
snapshot when getting promoted from inactive to active. This PR adds that
code. This fixes a bug where sometimes we would try to look for that in-memory
snapshot and not find it.
The controller always starts inactive, and there is no requirement that there
exists an in-memory snapshot at the last committed offset when the controller
is inactive. Therefore we can remove the initial snapshot at offset -1.
We should also optimize when a snapshot is cancelled or completes, by deleting
all in-memory snapshots less that the last committed offset.
SnapshotRegistry's createSnapshot should allow the creating of a snapshot if
the last snapshot's offset is the given offset. This allows for simpler client
code. Finally, this PR renames createSnapshot to getOrCreateSnapshot.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Also:
* Deprecate `log.message.format.version` and `message.format.version`.
* Log broker warning if the deprecated config values are ignored due to
the inter-broker protocol version.
* Log warning if `message.format.version` is set via `ConfigCommand`.
* Always down-convert if fetch version is v3 or lower.
* Add tests to verify new message format version based on the
inter-broker protocol version.
* Adjust existing tests that create topics with an older message format to
have the inter-broker protocol set to 2.8.
* Add upgrade note.
Note that the log compaction change to always write new segments with
record format v2 if the IBP is 3.0 or higher will be done as part of
KAFKA-13093 (with Kafka 3.1 as the target release version).
Reviewers: David Jacot <djacot@confluent.io>, David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
After noticing increased LISR times, we discovered a lot of time was spent synchronously flushing the partition metadata file. This PR changes the code so we asynchronously flush the files.
We ensure files are flushed before appending, renaming or closing the log to ensure we have the partition metadata information on disk. Three new tests have been added to address these cases.
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Jun Rao <junrao@gmail.com>
Updated FetchRequest and FetchResponse to use topic IDs rather than topic names.
Some of the complicated code is found in FetchSession and FetchSessionHandler.
We need to be able to store topic IDs and maintain a cache on the broker for IDs that may not have been resolved. On incremental fetch requests, we will try to resolve them or remove them if in toForget.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
This PR implements broker-side KRaft snapshots, including both saving and
loading. The code for triggering a periodic broker-side snapshot will come in a
follow-on PR. Loading should work with just this PR. It also implements
reloading broker snapshots after initialization.
In order to facilitate snapshots, this PR introduces the concept of
MetadataImage and MetadataDelta. MetadataImage represents the metadata state
retained in memory. It is basically a generalization of MetadataCache that
includes a few things that MetadataCache does not (such as features and client
quotas.) KRaftMetadataCache is now an accessor for the data stored in this object.
Similarly, MetadataImage replaces CacheConfigRespository and ClientQuotaCache.
It also subsumes kafka.server.metadata.MetadataImage and related classes.
MetadataDelta represents a change to a MetadataImage. When a KRaft snapshot is
loaded, we will accumulate all the changes into a MetadataDelta first, prior to
applying it. If we must reload a snapshot because we fell too far behind while
consuming metadata, the resulting MetadataDelta will contain all the changes
needed to catch us up. During normal operation, MetadataDelta is also used to
accumulate the changes of each incoming batch of metadata records. These
incremental deltas should be relatively small.
I have removed the logic for updating the various manager objects from
BrokerMetadataListener and placed it into BrokerMetadataPublisher. This makes
it easier to unit test BrokerMetadataListener.
Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
Use MockConfigRepository rather than CachedConfigRepository in unit
tests. This is useful for an upcoming change that will remove
CachedConfigRepository.
Reviewers: David Arthur <mumrah@gmail.com>
Use a caching `BufferSupplier` per request handler thread so that
decompression buffers are cached if supported by the underlying
`CompressionType`. This achieves a similar outcome as #9220, but
with less contention.
We introduce a `RequestLocal` class to make it easier to introduce
new request scoped stateful instances (one example we discussed
previously was an `ActionQueue` that could be used to avoid
some of the complex group coordinator locking).
This is a small win for zstd (no synchronization or soft references) and
a more significant win for lz4. In particular, it reduces allocations
significantly when the number of partitions is high. The decompression
buffer size is typically 64 KB, so a produce request with 1000 partitions
results in 64 MB of allocations even if each produce batch is small (likely,
when there are so many partitions).
I did a quick producer perf local test with 5000 partitions, 1 KB record
size,
1 broker, lz4 and ~0.5 for the producer compression rate metric:
Before this change:
> 20000000 records sent, 346314.349535 records/sec (330.27 MB/sec),
148.33 ms avg latency, 2267.00 ms max latency, 115 ms 50th, 383 ms 95th, 777 ms 99th, 1514 ms 99.9th.
After this change:
> 20000000 records sent, 431956.113259 records/sec (411.95 MB/sec),
117.79 ms avg latency, 1219.00 ms max latency, 99 ms 50th, 295 ms 95th, 440 ms 99th, 662 ms 99.9th.
That's a 25% throughput improvement and p999 latency was reduced to
under half (in this test).
Default arguments will be removed in a subsequent PR.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This patch adds support for running the ZooKeeper-based
kafka.security.authorizer.AclAuthorizer with KRaft clusters. Set the
authorizer.class.name config as well as the zookeeper.connect config while also
setting the typical KRaft configs (node.id, process.roles, etc.), and the
cluster will use KRaft for metadata and ZooKeeper for ACL storage. A system
test that exercises the authorizer is included.
This patch also changes "Raft" to "KRaft" in several system test files. It also
fixes a bug where system test admin clients were unable to connect to a cluster
with broker credentials via the SSL security protocol when the broker was using
that for inter-broker communication and SASL for client communication.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
Support sorting the elements in ImplicitLinkedHashCollection.
This is useful sometimes in unit tests for comparing collections.
Reviewers: Ismael Juma <ismael@juma.me.uk>
KIP-516 introduced partition.metadata file to persist the topic ID on the broker. It is created through handling the LeaderAndIsrRequest in ReplicaManager. (See https://github.com/apache/kafka/pull/10143 for the code path.) RaftReplicaManager was missing the analogue code path for Kip-500 code. Like in ReplicaManager, RaftReplicaManager will now check the partition.metadata file when handling metadata records.
However, since we know that all raft topics will have topic IDs, we can simply set the ID in the log upon the log's creation.
Updated the ReplicaManager path to do the same on newly created topics.
There are also some tweaks to the checking logic to better handle the scenario when the log exists but is not yet associated to Partition (for example, upon startup after a shutdown).
Tests added to ensure the file is created and that the correct error is thrown when the id is inconsistent.
Added tests for creating the log with the new topic ID parameter.
Also adds a few methods to get topic ID from MetadataImageBuilder as this is the most convenient way to get topic ID from RaftReplicaManager.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Jason Gustafson <jason@confluent.io>
Gradle 7.0 is required for Java 16 compatibility and it removes a number of
deprecated APIs. Fix most issues preventing the upgrade to Gradle 7.0.
The remaining ones are more complicated and should be handled
in a separate PR. Details of the changes:
* Release tarball no longer includes includes test, sources, javadoc and test sources jars (these
are still published to the Maven Central repository).
* Replace `compile` with `api` or `implementation` - note that `implementation`
dependencies appear with `runtime` scope in the pom file so this is a (positive)
change in behavior
* Add missing dependencies that were uncovered by the usage of `implementation`
* Replace `testCompile` with `testImplementation`
* Replace `runtime` with `runtimeOnly` and `testRuntime` with `testRuntimeOnly`
* Replace `configurations.runtime` with `configurations.runtimeClasspath`
* Replace `configurations.testRuntime` with `configurations.testRuntimeClasspath` (except for
the usage in the `streams` project as that causes a cyclic dependency error)
* Use `java-library` plugin instead of `java`
* Use `maven-publish` plugin instead of deprecated `maven` plugin - this changes the
commands used to publish and to install locally, but task aliases for `install` and
`uploadArchives` were added for backwards compatibility
* Removed `-x signArchives` line from the readme since it was wrong (it was a
no-op before and it fails now, however)
* Replaces `artifacts` block with an approach that works with the `maven-publish` plugin
* Don't publish `jmh-benchmark` module - the shadow jar is pretty large and not
particularly useful (before this PR, we would publish the non shadow jars)
* Replace `version` with `archiveVersion`, `baseName` with `archiveBaseName` and
`classifier` with `archiveClassifier`
* Update Gradle and plugins to the latest stable version (7.0 is not stable yet)
* Use `plugin` DSL to configure plugins
* Updated notable changes for 3.0
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Randall Hauch <rhauch@gmail.com>
1. rename INVALID_HIGHWATERMARK to INVALID_HIGH_WATERMARK
2. replace FetchResponse.AbortedTransaction by FetchResponseData.AbortedTransaction
3. remove redundant constructors from FetchResponse.PartitionData
4. rename recordSet to records
5. add helpers "recordsOrFail" and "recordsSize" to FetchResponse to process record casting
Reviewers: Ismael Juma <ismael@juma.me.uk>
Previously all APIs were accessible on every listener exposed by the broker, but
with KIP-500, that is no longer true. We now have more complex requirements for
API accessibility.
For example, the KIP-500 controller exposes some APIs which are not exposed by
brokers, such as BrokerHeartbeatRequest, and does not expose most client APIs,
such as JoinGroupRequest, etc. Similarly, the KIP-500 broker does not implement
some APIs that the ZK-based broker does, such as LeaderAndIsrRequest and
UpdateFeaturesRequest.
All of this means that we need more sophistication in how we expose APIs and
keep them consistent with the ApiVersions API. Up until now, we have been
working around this using the controllerOnly flag inside ApiKeys, but this is
not rich enough to support all of the cases listed above. This PR introduces a
new "listeners" field to the request schema definitions. This field is an array
of strings which indicate the listener types in which the API should be exposed.
We currently support "zkBroker", "broker", and "controller". ("broker"
indicates the KIP-500 broker, whereas zkBroker indicates the old broker).
This PR also creates ApiVersionManager to encapsulate the creation of the
ApiVersionsResponse based on the listener type. Additionally, it modifies
SocketServer to check the listener type of received requests before forwarding
them to the request handler.
Finally, this PR also fixes a bug in the handling of the ApiVersionsResponse
prior to authentication. Previously a static response was sent, which means that
changes to features would not get reflected. This also meant that the logic to
ensure that only the intersection of version ranges supported by the controller
would get exposed did not work. I think this is important because some clients
rely on the initial pre-authenticated ApiVersions response rather than doing a
second round after authentication as the Java client does.
One final cleanup note: I have removed the expectation that envelope requests
are only allowed on "privileged" listeners. This made sense initially because
we expected to use forwarding before the KIP-500 controller was available. That
is not the case anymore and we expect the Envelope API to only be exposed on the
controller listener. I have nevertheless preserved the existing workarounds to
allow verification of the forwarding behavior in integration testing.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
This PR adds the KIP-500 BrokerServer and ControllerServer classes and
makes some related changes to get them working. Note that the ControllerServer
does not instantiate a QuorumController object yet, since that will be added in
PR #10070.
* Add BrokerServer and ControllerServer
* Change ApiVersions#computeMaxUsableProduceMagic so that it can handle
endpoints which do not support PRODUCE (such as KIP-500 controller nodes)
* KafkaAdminClientTest: fix some lingering references to decommissionBroker
that should be references to unregisterBroker.
* Make some changes to allow SocketServer to be used by ControllerServer as
we as by the broker.
* We now return a random active Broker ID as the Controller ID in
MetadataResponse for the Raft-based case as per KIP-590.
* Add the RaftControllerNodeProvider
* Add EnvelopeUtils
* Add MetaLogRaftShim
* In ducktape, in config_property.py: use a KIP-500 compatible cluster ID.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
We don't really need it and it causes problems in older Android versions
and GraalVM native image usage (there are workarounds for the latter).
Move the logic to separate classes that are only invoked when the
relevant compression library is actually used. Place such classes
in their own package and enforce via checkstyle that only these
classes refer to compression library packages.
To avoid cyclic dependencies, moved `BufferSupplier` to the `utils`
package.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Currently the partition.metadata file is created when the log is created. However, clusters with older inter-broker protocols will never use this file. This PR moves the creation of the file to when we write to the file.
This PR also deletes the partition.metadata file on startup if the IBP version is lower than 2.8.
Reviewers: Jun Rao <junrao@gmail.com>
Currently log recovery begins as soon as we instantiate `LogManager`, but when using a
Raft-based metadata quorum we won't have configs until after we catch up on the metadata
log. We therefore defer log recovery until we actually invoke `startup()` on the `LogManager`
instance. This timing difference has no effect when using ZooKeeper because we
immediately invoke `startup()` on the instantiated instance, but it gives us the necessary
flexibility for accurate log recovery with updated configs when using a Raft-based metadata
quorum.
The `LogCleaner` is currently instantiated during construction just after log recovery
completes, and then it is started in `startup()`. As an extra precaution, since we are
no longer performing recovery during construction, we both instantiate and start the
log cleaner in `startup()` after log recovery completes.
We also convert `LogManager` to use a `ConfigRepository` to load topic configs
(which can override the default log configs) instead of having a hard-coded
dependency on ZooKeeper. We retrieve the topic configs when we invoke `startup()`
-- which again is effectively no different from a timing perspective than what we do
today for the ZooKeeper case.
One subtlety is that currently we create the log configs for every topic at this point
-- if a topic has no config overrides then we associate a copy of the default
configuration with the topic inside a map, and we retrieve the log configs for that
topic's partitions from from that map during recovery. This PR makes a change to
this series of events as follows. We do not associate a copy of the the default
configuration with a topic in the map if the topic has no configs set when we query
for them. This saves some memory -- we don't unnecessarily copy the default
config many times -- but it also means we have use the default log configs for
that topic later on when recovery for each of its partitions begins.
The difference is that the default configs are dynamically reconfigurable, and they
could potentially change between the time when we invoke `startup()` and when
log recovery begins (log recovery can begin quite some time after `startup()` is
invoked if shutdown was unclean). Prior to this patch such a change would not
be used; with this patch they could be if they happen before recovery begins.
This actually is better -- we are performing log recovery with the most recent
known defaults when a topic had no overrides at all. Also, `Partition.createLog`
has logic to handle missed config updates, so the behavior is eventually the same.
The transition of the broker state from `STARTING` to `RECOVERY` currently
happens within the `LogManager`, and it only occurs if the shutdown was
unclean. We move this transition into the broker as it avoids passing a
reference to the broker state into the `LogManager`. We also now always
transition the broker into the `RECOVERY` state as dictated by [the KIP-631 broker state machine](https://cwiki.apache.org/confluence/display/KAFKA/KIP-631%3A+The+Quorumbased+Kafka+Controller#KIP631:TheQuorumbasedKafkaController-TheBrokerStateMachine).
Finally, a few clean-ups were included. One worth highlighting is that `Partition`
no longer requires a `ConfigRepository`.
Reviewers: David Arthur <david.arthur@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Consolidate auto topic creation logic to either forward a CreateTopicRequest or handling the creation directly as AutoTopicCreationManager, when handling FindCoordinator/Metadata request.
Co-authored-by: Jason Gustafson <jason@confluent.io>
Reviewers: Jason Gustafson <jason@confluent.io>
Refactor the MetadataCache into two implementations that both implement a common trait. This will let us
continue to use the existing implementation when using ZK, but use the new implementation when in kip-500 mode.
Reviewers: Colin McCabe <cmccabe@apache.org>, Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
`KafkaApis` is configured differently when it is used in a broker with a Raft-based controller quorum vs. a ZooKeeper quorum. For example, when using Raft, `ForwardingManager` is required rather than optional, and there is no `AdminManager`, `KafkaController`, or `KafkaZkClient`. This PR introduces `MetadataSupport` to abstract the two possibilities: `ZkSupport` and `RaftSupport`. This provides a fluent way to decide what to do based on the type of support that `KafkaApis` has been configured with. Certain types of requests are not supported when using raft (`AlterIsrRequest`, `UpdateMetadataRequest`, etc.), and `MetadataSupport` gives us an intuitive way to identify the constraints and requirements associated with the different configurations and react accordingly.
Existing tests are sufficient to detect bugs and regressions.
Reviewers: José Armando García Sancio <jsancio@gmail.com>, Jason Gustafson <jason@confluent.io>
`Partition` objects are able to retrieve topic configs when creating their log, and currently they do so with an implementation of `trait TopicConfigFetcher` that uses ZooKeeper. ZooKeeper is not available when using a Raft-based metadata log, so we need to abstract the retrieval of configs so it can work either with or without ZooKeeper. This PR introduces `trait ConfigRepository` with `ZkConfigRepository` and `CachedConfigRepository` implementations. `Partition` objects now use a provided `ConfigRepository` to retrieve topic configs, and we eliminate `TopicConfigFetcher` as it is no longer needed.
`ReplicaManager` now contains an instance of `ConfigRepository` so it can provide it when creating `Partition` instances.
`KafkaApis` needs to be able to handle describe-config requests; it currently delegates that to `ZkAdminManager`, which of course queries ZooKeeper. To make this work with or without ZooKeeper we move the logic from `ZkAdminManager` into a new `ConfigHelper` class that goes through a `ConfigRepository` instance. We provide `KafkaApis` with such an instance, and it creates an instance of the helper so it can use that instead of going through `ZkAdminManager`.
Existing tests are sufficient to identify bugs and regressions in `Partition`, `ReplicaManager`, `KafkaApis`, and `ConfigHelper`. The `ConfigRepository` implementations have their own unit tests.
Reviewers: Jason Gustafson <jason@confluent.io>
Replace BrokerStates.scala with BrokerState.java, to make it easier to use from Java code if needed. This also makes it easier to go from a numeric type to an enum.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
The primary purpose of this patch is to remove the internal `enable.metadata.quorum` configuration. Instead, we rely on `process.roles` to determine if the self-managed quorum has been enabled. As a part of this, I've done the following:
1. Replace the notion of "disabled" APIs with "controller-only" APIs. We previously marked some APIs which were intended only for the KIP-500 as "disabled" so that they would not be unintentionally exposed. For example, the Raft quorum APIs were disabled. Marking them as "controller-only" carries the same effect, but makes the intent that they should be only exposed by the KIP-500 controller clearer.
2. Make `ForwardingManager` optional in `KafkaServer` and `KafkaApis`. Previously we used `null` if forwarding was enabled and relied on the metadata quorum check.
3. Make `zookeeper.connect` an optional configuration if `process.roles` is defined.
4. Update raft README to remove reference to `zookeeper.conntect`
Reviewers: Colin Patrick McCabe <cmccabe@confluent.io>, Boyang Chen <boyang@confluent.io>
Rename AdminManager to ZkAdminManager to emphasize the fact that it is not used by the KIP-500 code paths.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Boyang Chen <boyang@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
ISR-related cleanup in ReplicaManager and Partition. Removes ISR change logic from ReplicaManager and adds a new ZkIsrManager class which adheres to a new AlterIsrManager trait. Unifies all of the ISR logic in Partition so we don't have separate code paths for ZK vs AlterIsr. Also removes PartitionStateStore
Includes:
- Bump the version of MetadataRequest and MetadataResponse, add topicId in MetadataResponse
- Alter describeTopic in AdminClientTopicService and ZookeeperTopicService
- TopicMetadata is cached in MetadataCache, so we need to add topicId to MetadataCache
- MetadataCache is updated by UpdateMetadataRequest, bump the version of UpdateMetadataReq and UpdateMetadataResp, add topicId in UpdateMetadataReq.
Reviewers: Justine Olshan <jolshan@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
Includes:
- New API to authorize by resource type
- Default implementation for the method that supports super users and ACLs
- Optimized implementation in AclAuthorizer that supports ACLs, super users and allow.everyone.if.no.acl.found
- Benchmarks and tests
- InitProducerIdRequest authorized for Cluster:IdempotentWrite or WRITE to any topic, ProduceRequest authorized only for topic even if idempotent
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
This patch follows up https://github.com/apache/kafka/pull/9547. It refactors AbstractFetcherThread and its descendants to use `OffsetForLeaderEpochRequestData.OffsetForLeaderPartition` instead of `OffsetsForLeaderEpochRequest.PartitionData`. The patch relies on existing tests.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
* The naming for `ListOffsets` was inconsistent, in some places it was `ListOffset` and in others
it was `ListOffsets`. Picked the latter since it was used in metrics and the protocol documentation
and made it consistent.
* Removed unused methods in ApiKeys.
* Deleted `CommonFields`.
* Added `lowestSupportedVersion` and `highestSupportedVersion` to `ApiMessageType`
* Removed tests in `MessageTest` that are no longer relevant.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This patch updates the request logger to output request and response payloads in JSON. Payloads are converted to JSON based on their auto-generated schema.
Reviewers: Lucas Bradstreet <lucas@confluent.io>, David Mao <dmao@confluent.io>, David Jacot <djacot@confluent.io>
Also updated the jmh readme to make it easier for new people to know
what's possible and best practices.
There were some changes in the generated benchmarking code that
required adjusting `spotbugs-exclude.xml` and for a `javac` warning
to be suppressed for the benchmarking module. I took the chance
to make the spotbugs exclusion mode maintainable via a regex
pattern.
Tested the commands on Linux and macOS with zsh.
JMH highlights:
* async-profiler integration. Can be used with -prof async,
pass -prof async:help to look for the accepted options.
* perf c2c [2] integration. Can be used with -prof perfc2c,
if available.
* JFR profiler integration. Can be used with -prof jfr, pass
-prof jfr:help to look for the accepted options.
Full details:
* 1.24: https://mail.openjdk.java.net/pipermail/jmh-dev/2020-August/002982.html
* 1.25: https://mail.openjdk.java.net/pipermail/jmh-dev/2020-August/002987.html
* 1.26: https://mail.openjdk.java.net/pipermail/jmh-dev/2020-October/003024.html
* 1.27: https://mail.openjdk.java.net/pipermail/jmh-dev/2020-December/003096.html
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Lucas Bradstreet <lucasbradstreet@gmail.com>
Connection id is now only present in `NetworkSend`, which is now
the class used by `Selector`/`NetworkClient`/`KafkaChannel` (which
works well since `NetworkReceive` is the class used for
received data).
The previous `NetworkSend` was also responsible for adding a size
prefix. This logic is already present in `SendBuilder`, but for the
minority of cases where `SendBuilder` is not used (including
a number of tests), we now have `ByteBufferSend.sizePrefixed()`.
With regards to the request/message utilities:
* Renamed `toByteBuffer`/`toBytes` in `MessageUtil` to
`toVersionPrefixedByteBuffer`/`toVersionPrefixedBytes` for clarity.
* Introduced new `MessageUtil.toByteBuffer` that does not include
the version as the prefix.
* Renamed `serializeBody` in `AbstractRequest/Response` to
`serialize` for symmetry with `parse`.
* Introduced `RequestTestUtils` and moved relevant methods from
`TestUtils`.
* Moved `serializeWithHeader` methods that were only used in
tests to `RequestTestUtils`.
* Deleted `MessageTestUtil`.
Finally, a couple of changes to simplify coding patterns:
* Added `flip()` and `buffer()` to `ByteBufferAccessor`.
* Added `MessageSizeAccumulator.sizeExcludingZeroCopy`.
* Used lambdas instead of `TestCondition`.
* Used `Arrays.copyOf` instead of `System.arraycopy` in `MessageUtil`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
Generated request/response classes have code to serialize/deserialize directly to
`ByteBuffer` so the intermediate conversion to `Struct` can be skipped for them.
We have recently completed the transition to generated request/response classes,
so we can also remove the `Struct` based fallbacks.
Additional noteworthy changes:
* `AbstractRequest.parseRequest` has a more efficient computation of request size that
relies on the received buffer instead of the parsed `Struct`.
* Use `SendBuilder` for `AbstractRequest/Response` `toSend`, made the superclass
implementation final and removed the overrides that are no longer necessary.
* Removed request/response constructors that assume latest version as they are unsafe
outside of tests.
* Removed redundant version fields in requests/responses.
* Removed unnecessary work in `OffsetFetchResponse`'s constructor when version >= 2.
* Made `AbstractResponse.throttleTimeMs()` abstract.
* Using `toSend` in `SaslClientAuthenticator` instead of `serialize`.
* Various changes in Request/Response classes to make them more consistent and to
rely on the Data classes as much as possible when it comes to their state.
* Remove the version argument from `AbstractResponse.toString`.
* Fix `getErrorResponse` for `ProduceRequest` and `DescribeClientQuotasRequest` to
use `ApiError` which processes the error message sent back to the clients. This was
uncovered by an accidental fix to a `RequestResponseTest` test (it was calling
`AbstractResponse.toString` instead of `AbstractResponse.toString(short)`).
Rely on existing protocol tests to ensure this refactoring does not change
observed behavior (aside from improved performance).
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Add small interface to Partition.scala that allows AlterIsr and ZK code paths to update the ISR metrics managed by ReplicaManager. This opens the door for consolidating even more code between the two ISR update code paths.
This patch follows up https://github.com/apache/kafka/pull/9547. It refactors KafkaApis, ReplicaManager and Partition to use `OffsetForLeaderEpochResponseData.EpochEndOffset` instead of `EpochEndOffset`. In the mean time, it removes `OffsetsForLeaderEpochRequest#epochsByTopicPartition` and `OffsetsForLeaderEpochResponse#responses` and replaces their usages to use the automated protocol directly. Finally, it removes old constructors in `OffsetsForLeaderEpochResponse`. The patch relies on existing tests.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
From IBP 2.7 onwards, fetch responses include diverging epoch and offset in fetch responses if lastFetchedEpoch is provided in the fetch request. This PR uses that information for truncation and avoids the additional OffsetForLeaderEpoch requests in followers when lastFetchedEpoch is known.
Co-authored-by: Jason Gustafson <jason@confluent.io>
Reviewers: Jason Gustafson <jason@confluent.io>, Nikhil Bhatia <rite2nikhil@gmail.com>
This patch changes the grouping of `Send` objects created by `SendBuilder` in order to reduce the number of generated `Send` objects and thereby the number of system writes.
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
This PR adds support for forwarding of the following RPCs:
AlterConfigs
IncrementalAlterConfigs
AlterClientQuotas
CreateTopics
Co-authored-by: Jason Gustafson <jason@confluent.io>
Reviewers: Jason Gustafson <jason@confluent.io>
This reverts commit 21dc5231ce as we decide to use Envelope for redirection instead of initial principal.
Reviewers: Jason Gustafson <jason@confluent.io>
Summary:
In this PR, I have implemented the write path of the feature versioning system (KIP-584). Here is a summary of what's in this PR:
New APIs in org.apache.kafka.clients.admin.Admin interface, and their client and server implementations. These APIs can be used to describe features and update finalized features. These APIs are: Admin#describeFeatures and Admin#updateFeatures.
The write path is provided by the Admin#updateFeatures API. The corresponding server-side implementation is provided in KafkaApis and KafkaController classes. This can be a good place to start the code review.
The write path is supplemented by Admin#describeFeatures client API. This does not translate 1:1 to a server-side API. Instead, under the hood the API makes an explicit ApiVersionsRequest to the Broker to fetch the supported and finalized features.
Implemented a suite of integration tests in UpdateFeaturesTest.scala that thoroughly exercises the various cases in the write path.
Other changes:
The data type of the FinalizedFeaturesEpoch field in ApiVersionsResponse has been modified from int32 to int64. This change is to conform with the latest changes to the KIP explained in the voting thread.
Along the way, the class SupportedFeatures has been renamed to be called BrokerFeatures, and, it now holds both supported features as well as default minimum version levels.
For the purpose of testing, both the BrokerFeatures and FinalizedFeatureCache classes have been changed to be no longer singleton in implementation. Instead, these are now instantiated once and maintained in KafkaServer. The singleton instances are passed around to various classes, as needed.
Reviewers: Boyang Chen <boyang@confluent.io>, Jun Rao <junrao@gmail.com>
There are no checks on the header key so instantiating key (bytes to string) is unnecessary.
One implication is that conversion failures will be detected a bit later, but this is consistent
with how we handle the header value.
**JMH RESULT**
1. ops: +12%
1. The optimization of memory usage is very small as the cost of creating extra ```ByteBuffer``` is
almost same to byte array copy (used to construct ```String```). Using large key results in better
improvement but I don't think large key is common case.
**BEFORE**
```
Benchmark (bufferSupplierStr) (bytes) (compressionType) (headerKeySize) (maxBatchSize) (maxHeaderSize) (messageSize) (messageVersion) Mode Cnt Score Error Units
RecordBatchIterationBenchmark.measureValidation NO_CACHING RANDOM NONE 10 200 5 1000 2 thrpt 15 2035938.174 ± 1653.566 ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm NO_CACHING RANDOM NONE 10 200 5 1000 2 thrpt 15 2040.000 ± 0.001 B/op
```
```
Benchmark (bufferSupplierStr) (bytes) (compressionType) (headerKeySize) (maxBatchSize) (maxHeaderSize) (messageSize) (messageVersion) Mode Cnt Score Error Units
RecordBatchIterationBenchmark.measureValidation NO_CACHING RANDOM NONE 30 200 5 1000 2 thrpt 15 1979193.376 ± 1239.286 ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm NO_CACHING RANDOM NONE 30 200 5 1000 2 thrpt 15 2120.000 ± 0.001 B/op
```
**AFTER**
```
Benchmark (bufferSupplierStr) (bytes) (compressionType) (headerKeySize) (maxBatchSize) (maxHeaderSize) (messageSize) (messageVersion) Mode Cnt Score Error Units
RecordBatchIterationBenchmark.measureValidation NO_CACHING RANDOM NONE 10 200 5 1000 2 thrpt 15 2289115.973 ± 2661.856 ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm NO_CACHING RANDOM NONE 10 200 5 1000 2 thrpt 15 2032.000 ± 0.001 B/op
```
```
Benchmark (bufferSupplierStr) (bytes) (compressionType) (headerKeySize) (maxBatchSize) (maxHeaderSize) (messageSize) (messageVersion) Mode Cnt Score Error Units
RecordBatchIterationBenchmark.measureValidation NO_CACHING RANDOM NONE 30 200 5 1000 2 thrpt 15 2222625.706 ± 908.358 ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm NO_CACHING RANDOM NONE 30 200 5 1000 2 thrpt 15 2040.000 ± 0.001 B/op
```
Reviewers: Ismael Juma <ismael@juma.me.uk>
Only check if positions need validation if there is new metadata.
Also fix some inefficient java.util.stream code in the hot path of SubscriptionState.
Refactored FetchRequest and FetchResponse to use the generated message classes for serialization and deserialization. This allows us to bypass unnecessary Struct conversion in a few places. A new "records" type was added to the message protocol which uses BaseRecords as the field type. When sending, we can set a FileRecords instance on the message, and when receiving the message class will use MemoryRecords.
Also included a few JMH benchmarks which indicate a small performance improvement for requests with high partition counts or small record sizes.
Reviewers: Jason Gustafson <jason@confluent.io>, Boyang Chen <boyang@confluent.io>, David Jacot <djacot@confluent.io>, Lucas Bradstreet <lucas@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>
Set `replica.fetch.max.bytes` to `1` and produce multiple record batches to allow
for throttling to take place. This helps avoid a race condition where the
reassignment would complete more quickly than expected causing an
assertion to fail some times.
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Jason Gustafson <jason@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
This PR implements the broker side changes of KIP-599, except the changes of the Rate implementation which will be addressed separately. The PR changes/introduces the following:
- It introduces the protocol changes.
- It introduces a new quota manager ControllerMutationQuotaManager which is another specialization of the ClientQuotaManager.
- It enforces the quota in the KafkaApis and in the AdminManager. This part handles new and old clients as described in the KIP.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
For KIP-392, we added logic to make sure that high watermark changes are propagated to followers without delay in order to improve end to end latency when fetching from followers. The downside of this change is that it can increase the rate of fetch requests from followers which can have a noticeable impact on performance (see KAFKA-9731).
To fix that problem, we have previously modified the code so that we only propagate high watermark changes immediately when a replica selector is used (which is not the default). However, leaving this logic around means that it is risky to enable follower fetching since it changes the follower request rate, which can have a big impact on overall broker performance.
This patch disables immediate propagation of the high watermark more generally. Instead, users can use the max wait time in order to control the worst-case latency. Note that this is typically only a problem anyway for low-throughput clusters since otherwise we will always have a steady rate of high watermark updates.
Reviewers: Ismael Juma <ismael@juma.me.uk>
KAFKA-7283 enabled lazy mmap on index files by initializing indices
on-demand rather than performing costly disk/memory operations when
creating all indices on broker startup. This helped reducing the startup
time of brokers. However, segment indices are still created on closing
segments, regardless of whether they need to be closed or not.
This is a cleaned up version of #7900, which was submitted by @efeg. It
eliminates unnecessary disk accesses and memory map operations while
deleting, renaming or closing offset and time indexes.
In a cluster with 31 brokers, where each broker has 13K to 20K segments,
@efeg and team observed up to 2 orders of magnitude faster LogManager
shutdown times - i.e. dropping the LogManager shutdown time of each
broker from 10s of seconds to 100s of milliseconds.
To avoid confusion between `renameTo` and `setFile`, I replaced the
latter with the more restricted updateParentDir` (it turns out that's
all we need).
Reviewers: Jun Rao <junrao@gmail.com>, Andrew Choi <a24choi@edu.uwaterloo.ca>
Co-authored-by: Adem Efe Gencer <agencer@linkedin.com>
Co-authored-by: Ismael Juma <ismael@juma.me.uk>
This PR works to improve high watermark checkpointing performance.
`ReplicaManager.checkpointHighWatermarks()` was found to be a major contributor to GC pressure, especially on Kafka clusters with high partition counts and low throughput.
Added a JMH benchmark for `checkpointHighWatermarks` which establishes a
performance baseline. The parameterized benchmark was run with 100, 1000 and
2000 topics.
Modified `ReplicaManager.checkpointHighWatermarks()` to avoid extra copies and cached
the Log parent directory Sting to avoid frequent allocations when calculating
`File.getParent()`.
A few clean-ups:
* Changed all usages of Log.dir.getParent to Log.parentDir and Log.dir.getParentFile to
Log.parentDirFile.
* Only expose public accessor for `Log.dir` (consistent with `Log.parentDir`)
* Removed unused parameters in `Partition.makeLeader`, `Partition.makeFollower` and `Partition.createLogIfNotExists`.
Benchmark results:
| Topic Count | Ops/ms | MB/sec allocated |
|-------------|---------|------------------|
| 100 | + 51% | - 91% |
| 1000 | + 143% | - 49% |
| 2000 | + 149% | - 50% |
Reviewers: Lucas Bradstreet <lucas@confluent.io>. Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Gardner Vickers <gardner@vickers.me>
Co-authored-by: Ismael Juma <ismael@juma.me.uk>
This PR removes intermediate conversions between `MetadataResponse.TopicMetadata` => `MetadataResponseTopic` and `MetadataResponse.PartitionMetadata` => `MetadataResponsePartition` objects.
There is 15-20% reduction in object allocations and 5-10% improvement in metadata request performance.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson<jason@confluent.io>
To dismiss the usage of operation ++ against Set which is slow when Set has many entries. This pr introduces a new class 'AclSets' which takes multiple Sets as parameters and do 'find' against them one by one. For more details about perf and benchmark, refer to [KAFKA-9685](https://issues.apache.org/jira/browse/KAFKA-9685)
Author: jiao <jiao.zhang@linecorp.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8261 from jiao-zhangS/jira-9685
This PR adds new lock is used to prevent the follower replica from being updated while ReplicaAlterDirThread is executing maybeReplaceCurrentWithFutureReplica() to replace follower replica with the future replica.
Now doAppendRecordsToFollowerOrFutureReplica() doesn't need to hold the lock on leaderIsrUpdateLock for local replica updation and ongoing log appends on the follower will not delay the makeFollower() call.
**Benchmark results for Partition.makeFollower() **
Old:
```
Benchmark Mode Cnt Score Error Units
PartitionMakeFollowerBenchmark.testMakeFollower avgt 15 2046.967 ? 22.842 ns/op
```
New:
```
Benchmark Mode Cnt Score Error Units
PartitionMakeFollowerBenchmark.testMakeFollower avgt 15 1278.525 ? 5.354 ns/op
```
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#8153 from omkreddy/KAFKA-9594-LAISR
This change fixes a performance regression due to follower last seen highwatermark
handling introduced in 23beeea. maybeUpdateHwAndSendResponse is expensive for
brokers with high partition counts, as it requires a partition and a replica lookup for every
partition being fetched. This refactor moves the last seen watermark update into the follower
fetch state update where we have already looked up the partition and replica.
I've seen cases where maybeUpdateHwAndSendResponse is responsible 8% of CPU usage, not including the responseCallback call that is part of it.
I have benchmarked this change with `UpdateFollowerFetchStateBenchmark` and it adds 5ns
of overhead to Partition.updateFollowerFetchState, which is a rounding error compared to the
current overhead of maybeUpdateHwAndSendResponse.
Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Replaced UpdateMetadata{Request, Response}, LeaderAndIsr{Request, Response}
and StopReplica{Request, Response} with the automated protocol classes.
Updated the JSON schema for the 3 request types to be more consistent and
less strict (if needed to avoid duplication).
The general approach is to avoid generating new collections in the request
classes. Normalization happens in the constructor to make this possible. Builders
still have to group by topic to maintain the external ungrouped view.
Introduced new tests for LeaderAndIsrRequest and UpdateMetadataRequest to
verify that the new logic is correct.
A few other clean-ups/fixes in code that was touched due to these changes:
* KAFKA-8956: Refactor DelayedCreatePartitions#updateWaiting to avoid modifying
collection in foreach.
* Avoid unnecessary allocation for state change trace logging if trace logging is not enabled
* Use `toBuffer` instead of `toList`, `toIndexedSeq` or `toSeq` as it generally performs
better and it matches the performance characteristics of `java.util.ArrayList`. This is
particularly important when passing such instances to Java code.
* Minor refactoring for clarity and readability.
* Removed usage of deprecated `/:`, unused imports and unnecessary `var`s.
* Include exception in `AdminClientIntegrationTest` failure message.
* Move StopReplicaRequest verification in `AuthorizerIntegrationTest` to the end
to match the comment.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
This PR makes two changes to code in the ReplicaManager.updateFollowerFetchState path, which is in the hot path for follower fetches. Although calling ReplicaManager.updateFollowerFetch state is inexpensive on its own, it is called once for each partition every time a follower fetch occurs.
1. updateFollowerFetchState no longer calls maybeExpandIsr when the follower is already in the ISR. This avoid repeated expansion checks.
2. Partition.maybeIncrementLeaderHW is also in the hot path for ReplicaManager.updateFollowerFetchState. Partition.maybeIncrementLeaderHW calls Partition.remoteReplicas four times each iteration, and it performs a toSet conversion. maybeIncrementLeaderHW now avoids generating any intermediate collections when updating the HWM.
**Benchmark results for Partition.updateFollowerFetchState on a r5.xlarge:**
Old:
```
1288.633 ±(99.9%) 1.170 ns/op [Average]
(min, avg, max) = (1287.343, 1288.633, 1290.398), stdev = 1.037
CI (99.9%): [1287.463, 1289.802] (assumes normal distribution)
```
New (when follower fetch offset is updated):
```
261.727 ±(99.9%) 0.122 ns/op [Average]
(min, avg, max) = (261.565, 261.727, 261.937), stdev = 0.114
CI (99.9%): [261.605, 261.848] (assumes normal distribution)
```
New (when follower fetch offset is the same):
```
68.484 ±(99.9%) 0.025 ns/op [Average]
(min, avg, max) = (68.446, 68.484, 68.520), stdev = 0.023
CI (99.9%): [68.460, 68.509] (assumes normal distribution)
```
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
The jmh LRUCacheBenchmark will exhibit an int overflow when run on a fast machine:
```
java.lang.ArrayIndexOutOfBoundsException: Index -3648 out of bounds for length 10000
at org.apache.kafka.jmh.cache.LRUCacheBenchmark.testCachePerformance(LRUCacheBenchmark.java:70)
at org.apache.kafka.jmh.cache.generated.LRUCacheBenchmark_testCachePerformance_jmhTest.testCachePerformance_thrpt_jmhStub(LRUCacheBenchmark_testCachePerformance_jmhTest.java:119)
at org.apache.kafka.jmh.cache.generated.LRUCacheBenchmark_testCachePerformance_jmhTest.testCachePerformance_Throughput(LRUCacheBenchmark_testCachePerformance_jmhTest.java:83)
```
Reviewers: Jason Gustafson <jason@confluent.io>
* KAFKA-8106:Reducing the allocation and copying of ByteBuffer when logValidator do validation.
* KAFKA-8106:Reducing the allocation and copying of ByteBuffer when logValidator do validation.
* github comments
* use batch.skipKeyValueIterator
* cleanups
* no need to skip kv for uncompressed iterator
* checkstyle fixes
* fix findbugs
* adding unit tests
* reuse decompression buffer; and using streaming iterator
* checkstyle
* add unit tests
* remove reusing buffer supplier
* fix unit tests
* add unit tests
* use streaming iterator
* minor refactoring
* rename
* github comments
* github comments
* reuse buffer at DefaultRecord caller
* some further optimization
* major refactoring
* further refactoring
* update comment
* github comments
* minor fix
* add jmh benchmarks
* update jmh
* github comments
* minor fix
* github comments
Verified that the https links work.
I didn't update the license header in this PR since that touches
so many files. Will file a separate one for that.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
- Use Xlint:all with 3 exclusions (filed KAFKA-7613 to remove the exclusions)
- Use the same javac options when compiling tests (seems accidental that
we didn't do this before)
- Replaced several deprecated method calls with non-deprecated ones:
- `KafkaConsumer.poll(long)` and `KafkaConsumer.close(long)`
- `Class.newInstance` and `new Integer/Long` (deprecated since Java 9)
- `scala.Console` (deprecated in Scala 2.11)
- `PartitionData` taking a timestamp (one of them seemingly a bug)
- `JsonMappingException` single parameter constructor
- Fix unnecessary usage of raw types in several places.
- Add @SuppressWarnings for deprecations, unchecked and switch fallthrough in
several places.
- Scala clean-ups (var -> val, ETA expansion warnings, avoid reflective calls)
- Use lambdas to simplify code in a few places
- Add @SafeVarargs, fix varargs usage and remove unnecessary `Utils.mkList` method
Reviewers: Matthias J. Sax <mjsax@apache.org>, Manikumar Reddy <manikumar.reddy@gmail.com>, Randall Hauch <rhauch@gmail.com>, Bill Bejeck <bill@confluent.io>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
Fixes two issues with the JMH benchmark example:
* Trivial: The output should be in `ops/ms` for readability
reasons (it's in the millions of operations per second)
* Important: The benchmark is not actually measuring the
LRU Cache performance as most of the time in each run is
wasted on concatenating `key + counter` as well as
`value + counter`. Fixed by pre-generating 10k K-V pairs
(100x the cache capacity) and iterating over them. This
brings the performance up by a factor of more than 5 on
a standard 4 core i7 (`~6k/ms` before goes to `~35k/ms`).
Author: Armin Braun <me@obrown.io>
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Guozhang Wang <wangguoz@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#2903 from original-brownbear/fix-jmh-example
…h-benchmarks/jmh.sh
Author: bbejeck <bbejeck@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
Closes#2654 from bbejeck/KAFKA-3989_follow_up
More specifically, fix the case where a compressed V0 or V1 message is
larger than the producer batch size.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Apurva Mehta <apurva@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Closes#3356 from hachikuji/KAFKA-5456
I included a JMH benchmark and the results follow. The
implementation in this PR takes no more than 1/10th
of the time when compared to trunk. I also included
results for an alternative implementation that is a little
slower than the one in the PR.
Trunk:
```text
TopicBenchmark.testValidate topic avgt 15 134.107 ± 3.956 ns/op
TopicBenchmark.testValidate longer-topic-name avgt 15 316.241 ± 13.379 ns/op
TopicBenchmark.testValidate very-long-topic-name_with_more_text avgt 15 636.026 ± 30.272 ns/op
```
Implementation in the PR:
```text
TopicBenchmark.testValidate topic avgt 15 13.153 ± 0.383 ns/op
TopicBenchmark.testValidate longer-topic-name avgt 15 26.139 ± 0.896 ns/op
TopicBenchmark.testValidate very-long-topic-name.with_more_text avgt 15 44.829 ± 1.390 ns/op
```
Alternative implementation where boolean validChar = Character.isLetterOrDigit(c) || c == '.' || c == '_' || c == '-';
```text
TopicBenchmark.testValidate topic avgt 15 18.883 ± 1.044 ns/op
TopicBenchmark.testValidate longer-topic-name avgt 15 36.696 ± 1.220 ns/op
TopicBenchmark.testValidate very-long-topic-name_with_more_text avgt 15 65.956 ± 0.669 ns/op
```
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Closes#3234 from ijuma/optimise-topic-is-valid
The JMH benchmark included shows that the redundant
volatile write causes the constructor of `ProducerRecord`
to take more than 50% longer:
ProducerRecordBenchmark.constructorBenchmark avgt 15 24.136 ± 1.458 ns/op (before)
ProducerRecordBenchmark.constructorBenchmark avgt 15 14.904 ± 0.231 ns/op (after)
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Jason Gustafson <jason@confluent.io>
Closes#3233 from ijuma/remove-volatile-write-in-records-header-constructor