Commit Graph

241 Commits

Author SHA1 Message Date
Nikolay 6f38fe5e0a
KAFKA-14588 ZK configuration moved to ZkConfig (#15075)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-27 22:37:01 +08:00
Jorge Esteban Quilcate Otoya b25c96a915
KAFKA-16229: Fix slow expired producer id deletion (#15324)
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>
2024-02-09 17:17:17 -08:00
David Arthur 7bf7fd99a5
KAFKA-16078: Be more consistent about getting the latest MetadataVersion
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>
2024-01-17 14:59:22 -08:00
Divij Vaidya 65424ab484
MINOR: New year code cleanup - include final keyword (#15072)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Sagar Rao <sagarmeansocean@gmail.com>
2024-01-11 17:53:35 +01:00
Fiore Mario Vitale 314de9f23c
KAFKA-15996: Improve JsonConverter performance (#14992)
Improve JsonConverter performance by using afterBurnModule of Jackson library.

Reviewers: Divij Vaidya <diviv@amazon.com>, Mickael Maison <mickael.maison@gmail.com>
2023-12-24 21:47:12 +01:00
Omnia Ibrahim 07490b929b
KAFKA-15365: Broker-side replica management changes (#14881)
Reviewers: Igor Soarez <soarez@apple.com>, Ron Dagostino <rndgstn@gmail.com>, Proven Provenzano <pprovenzano@confluent.io>
2023-12-11 09:34:22 -05:00
David Arthur a8622faf47
KAFKA-15799 Handle full metadata updates on ZK brokers (#14719)
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>
2023-11-16 14:38:44 -08:00
hudeqi 9911fab1a1
KAFKA-15432: RLM Stop partitions should not be invoked for non-tiered storage topics (#14667)
Reviewers: Christo Lolov <lolovc@amazon.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-11-02 10:00:15 +01:00
Calvin Liu 14029e2ddd
KAFKA-15582: Identify clean shutdown broker (#14465)
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>
2023-10-19 10:25:23 -07:00
Matthias J. Sax 9b468fb278
MINOR: Do not end Javadoc comments with `**/` (#14540)
Reviewers: Bruno Cadonna <bruno@confluent.io>, Bill Bejeck <bill@confluent.io>, Hao Li <hli@confluent.io>, Josep Prat <josep.prat@aiven.io>
2023-10-17 21:11:04 -07:00
Mehari Beyene 25b128de81
KAFKA-14991: KIP-937-Improve message timestamp validation (#14135)
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>
2023-08-24 12:04:55 +02:00
Luke Chen 748175ce62
KAFKA-15189: only init remote topic metrics when enabled (#14133)
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>
2023-08-05 13:00:16 +08:00
Luke Chen 27ea025e33
KAFKA-15176: add tests for tiered storage metrics (#13999)
Added tests for metrics:
1. RemoteLogReaderTaskQueueSize
2. RemoteLogReaderAvgIdlePercent
3. RemoteLogManagerTasksAvgIdlePercent

Also, added tests for OffsetOutOfRangeException will be thrown while reading logs

Reviewers: Christo Lolov <christololov@gmail.com>, Satish Duggana <satishd@apache.org>
2023-07-21 10:30:33 +08:00
Justine Olshan ea0bb00126
KAFKA-14884: Include check transaction is still ongoing right before append (take 2) (#13787)
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>
2023-07-14 15:18:11 -07:00
Colin P. McCabe cd3c0ab1a3 KAFKA-15060: fix the ApiVersionManager interface
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>
2023-06-19 16:46:44 -07:00
David Jacot 7eea2a3908
MINOR: Move MockTime to server-common (#13823)
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>
2023-06-09 08:54:25 +02:00
Divij Vaidya fe6a827e20
KAFKA-14633: Reduce data copy & buffer allocation during decompression (#13135)
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>
2023-06-05 15:04:49 +08:00
Yash Mayya 9bb2f78d53
KAFKA-15034: Improve performance of the ReplaceField SMT; add JMH benchmark (#13776)
Reviewers: Chris Egerton <chrise@aiven.io>
2023-06-01 15:14:31 -04:00
Justine Olshan 9edf2ec5cc
MINOR: Add transaction verification config to producerStateManager config (#13770)
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>
2023-05-30 13:46:17 -07:00
Divij Vaidya 6bcc497c36
KAFKA-14766: Improve performance of VarInt encoding and decoding (#13312)
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>
2023-05-05 20:05:20 +08:00
Luke Chen d796480fe8
KAFKA-14909: check zkMigrationReady tag before migration (#13631)
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>
2023-04-28 14:35:12 +08:00
Purshotam Chauhan df13775254
KAFKA-14828: Remove R/W locks using persistent data structures (#13437)
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>
2023-04-21 14:08:23 +05:30
Dimitar Dimitrov e14dd8024a
KAFKA-14821 Implement the listOffsets API with AdminApiDriver (#13432)
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>
2023-04-20 11:29:27 +08:00
Ron Dagostino e27926f92b
KAFKA-14735: Improve KRaft metadata image change performance at high … (#13280)
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>
2023-04-17 17:52:28 -04:00
Calvin Liu d5e216d618
KAFKA-14617: Fill broker epochs to the AlterPartitionRequest (#13489)
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>
2023-04-07 09:09:29 -07:00
Purshotam Chauhan f3e4dd9229
KAFKA-14827: Support for StandardAuthorizer benchmark (#13423)
* KAFKA-14827: Support for StandardAuthorizer benchmark

Co-authored-by: Purshotam Chauhan <purshotam.r.chauhan@gmail.com>

* reverting unintentional change

---------

Co-authored-by: David Arthur <mumrah@gmail.com>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2023-03-28 14:14:50 +05:30
Calvin Liu 79b5f7f1ce
KAFKA-14617: Add ReplicaState to FetchRequest (KIP-903) (#13323)
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>
2023-03-16 14:04:34 +01:00
Kowshik Prakasam 9f55945270
MINOR: Introduce OffsetAndEpoch in LeaderEndpoint interface return values (#13268)
Reviewers: Satish Duggana <satishd@apache.org>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Jun Rao <junrao@gmail.com>
2023-02-23 17:29:32 -08:00
Satish Duggana 069ce59e1e
KAFKA 14714: Move/Rewrite RollParams, LogAppendInfo, and LeaderHwChange to storage module. (#13255)
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2023-02-22 23:12:04 +05:30
David Jacot 3be7f7d611
KAFKA-14391; Add ConsumerGroupHeartbeat API (#12972)
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>
2023-02-09 09:13:31 +01:00
Satish Duggana 1d3fb76092
KAFKA-14688 Move package org.apache.kafka.server.log.internals to org.apache.kafka.storage.internals.log (#13213)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2023-02-08 09:22:42 +05:30
David Jacot 2e0a005dd4
KAFKA-14367; Add internal APIs to the new `GroupCoordinator` interface (#13112)
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>
2023-01-20 08:38:21 +01:00
Ismael Juma 8ac644d2b1
KAFKA-14607: Move Scheduler/KafkaScheduler to server-common (#13092)
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>
2023-01-10 23:51:58 -08:00
Akhilesh C db49070760
KAFKA-14493: Introduce Zk to KRaft migration state machine STUBs in KRaft controller. (#12998)
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>
2023-01-09 10:44:11 -08:00
Ismael Juma 96d9710c17
KAFKA-14478: Move LogConfig/CleanerConfig and related to storage module (#13049)
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>
2023-01-04 02:42:52 -08:00
Mickael Maison 8184ada6a5
KAFKA-14543: Move LogOffsetMetadata to storage module (#13038)
Reviewers: Ismael Juma <ismael@juma.me.uk>, dengziming <dengziming1993@gmail.com>, Satish Duggana <satishd@apache.org>, Federico Valeri <fedevaleri@gmail.com>
2022-12-27 17:12:02 -08:00
Federico Valeri 06af8fc630
KAFKA-14549: Move LogDirFailureChannel to storage module (#13041)
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>
2022-12-23 07:13:43 -08:00
Ismael Juma e8232edd24
KAFKA-14477: Move LogValidator and related to storage module (#13012)
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>
2022-12-21 16:57:02 -08:00
Ismael Juma 7b634c7034
KAFKA-14521: Replace BrokerCompressionCodec with BrokerCompressionType (#13011)
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>
2022-12-20 11:53:49 -08:00
Satish Duggana 7146ac57ba
[KAFKA-13369] Follower fetch protocol changes for tiered storage. (#11390)
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>
2022-12-17 09:36:44 -08:00
Akhilesh C 8b045dcbf6
KAFKA-14446: API forwarding support from zkBrokers to the Controller (#12961)
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>
2022-12-15 14:16:41 -08:00
Jeff Kim dbf5826cd5
KAFKA-14334: Complete delayed purgatory after replication (#12783)
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>
2022-11-16 14:54:42 +01:00
Justine Olshan a7caed39b7
KAFKA-14097: Make producer ID expiration a dynamic config (#12643)
Make `producer.id.expiration.ms` a dynamic configuration as described in KIP-854.

Reviewers: David Jacot <djacot@confluent.io>
2022-09-23 09:19:48 +02:00
Francesco Nigro aecd47b3bc
KAFKA-13900 Support Java 9 direct ByteBuffer Checksum methods (#12163)
Some numbers with JDK 11.

Before:
```
Benchmark                 (bytes)  (direct)  (readonly)  (seed)   Mode  Cnt   Score    Error   Units
Crc32CBenchmark.checksum      128     false       false      42  thrpt   20  26.730 ±  0.410  ops/us
Crc32CBenchmark.checksum      128      true       false      42  thrpt   20   1.781 ±  0.007  ops/us
Crc32CBenchmark.checksum     1024     false       false      42  thrpt   20   6.553 ±  0.053  ops/us
Crc32CBenchmark.checksum     1024      true       false      42  thrpt   20   0.223 ±  0.001  ops/us
Crc32CBenchmark.checksum     4096     false       false      42  thrpt   20   4.054 ±  0.015  ops/us
Crc32CBenchmark.checksum     4096      true       false      42  thrpt   20   0.056 ±  0.001  ops/us
```

And this PR:
```
Benchmark                 (bytes)  (direct)  (readonly)  (seed)   Mode  Cnt   Score   Error   Units
Crc32CBenchmark.checksum      128     false       false      42  thrpt   20  26.922 ± 0.065  ops/us
Crc32CBenchmark.checksum      128      true       false      42  thrpt   20  24.656 ± 0.620  ops/us
Crc32CBenchmark.checksum     1024     false       false      42  thrpt   20   6.548 ± 0.025  ops/us
Crc32CBenchmark.checksum     1024      true       false      42  thrpt   20   6.432 ± 0.136  ops/us
Crc32CBenchmark.checksum     4096     false       false      42  thrpt   20   4.031 ± 0.022  ops/us
Crc32CBenchmark.checksum     4096      true       false      42  thrpt   20   4.004 ± 0.016  ops/us
```

The purpose of the PR is to makes heap and direct ByteBuffer able to perform the same (especially
not read-only), without affecting the existing heap ByteBuffer performance.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Divij Vaidya <diviv@amazon.com>
2022-08-12 20:09:15 -07:00
Luke Chen 679e9e0cee
KAFKA-13919: expose log recovery metrics (#12347)
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>
2022-07-22 11:00:15 +08:00
David Arthur cc384054c6
KAFKA-13935 Fix static usages of IBP in KRaft mode (#12250)
* 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>
2022-06-13 14:23:28 -04:00
Rittika Adhikari 3467036e01
KAFKA-13803: Refactor Leader API Access (#12005)
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>
2022-06-03 09:12:06 -07:00
David Arthur 4efdc1a310
MINOR: Consolidate FinalizedFeatureCache into MetadataCache (#12214)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2022-05-26 16:25:58 -04:00
Jason Gustafson 8efdbce523
KAFKA-13837; Return an error from Fetch if follower is not a valid replica (#12150)
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>
2022-05-18 20:58:20 -07:00
Alyssa Huang 8245c9a3d5
KAFKA-13854 Refactor ApiVersion to MetadataVersion (#12072)
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>
2022-05-02 16:27:52 -07:00
Jason Gustafson f2a782a4d7
MINOR: Rename `AlterIsrManager` to `AlterPartitionManager` (#12089)
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>
2022-04-26 09:34:18 -07:00
David Jacot 7c8c65fc54
MINOR: Rename `ZkVersion` to `PartitionEpoch` (#12071)
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>
2022-04-22 20:38:17 +02:00
Lucas Bradstreet dc36dedd28
MINOR: jmh.sh swallows compile errors (#11870)
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>
2022-03-10 18:18:41 -05:00
Jason Koch c05403f47f
KAFKA-13629: Use faster algorithm for ByteUtils sizeOfXxx algorithm (#11721)
Replace loop with a branch-free implementation.

Include:
- Unit tests that includes old code and new code and runs through several ints/longs.
- JMH benchmark that compares old vs new performance of algorithm.

JMH results with JDK 17.0.2 and `compiler` blackhole mode are 2.8-3.4 faster with
the new implementation. In a real application, a 6% reduction in CPU cycles was
observed in the `send()` path via flamegraphs.

```
ByteUtilsBenchmark.testSizeOfUnsignedVarint                            thrpt    4  1472440.102 ±  67331.797  ops/ms
ByteUtilsBenchmark.testSizeOfUnsignedVarint:·async                     thrpt               NaN                  ---
ByteUtilsBenchmark.testSizeOfUnsignedVarint:·gc.alloc.rate             thrpt    4       ≈ 10⁻⁴               MB/sec
ByteUtilsBenchmark.testSizeOfUnsignedVarint:·gc.alloc.rate.norm        thrpt    4       ≈ 10⁻⁷                 B/op
ByteUtilsBenchmark.testSizeOfUnsignedVarint:·gc.count                  thrpt    4          ≈ 0               counts
ByteUtilsBenchmark.testSizeOfUnsignedVarintSimple                      thrpt    4   521333.117 ± 595169.618  ops/ms
ByteUtilsBenchmark.testSizeOfUnsignedVarintSimple:·async               thrpt               NaN                  ---
ByteUtilsBenchmark.testSizeOfUnsignedVarintSimple:·gc.alloc.rate       thrpt    4       ≈ 10⁻⁴               MB/sec
ByteUtilsBenchmark.testSizeOfUnsignedVarintSimple:·gc.alloc.rate.norm  thrpt    4       ≈ 10⁻⁶                 B/op
ByteUtilsBenchmark.testSizeOfUnsignedVarintSimple:·gc.count            thrpt    4          ≈ 0               counts
ByteUtilsBenchmark.testSizeOfVarlong                                   thrpt    4  1106519.633 ±  16556.502  ops/ms
ByteUtilsBenchmark.testSizeOfVarlong:·async                            thrpt               NaN                  ---
ByteUtilsBenchmark.testSizeOfVarlong:·gc.alloc.rate                    thrpt    4       ≈ 10⁻⁴               MB/sec
ByteUtilsBenchmark.testSizeOfVarlong:·gc.alloc.rate.norm               thrpt    4       ≈ 10⁻⁶                 B/op
ByteUtilsBenchmark.testSizeOfVarlong:·gc.count                         thrpt    4          ≈ 0               counts
ByteUtilsBenchmark.testSizeOfVarlongSimple                             thrpt    4   324435.607 ± 147754.813  ops/ms
ByteUtilsBenchmark.testSizeOfVarlongSimple:·async                      thrpt               NaN                  ---
ByteUtilsBenchmark.testSizeOfVarlongSimple:·gc.alloc.rate              thrpt    4       ≈ 10⁻⁴               MB/sec
ByteUtilsBenchmark.testSizeOfVarlongSimple:·gc.alloc.rate.norm         thrpt    4       ≈ 10⁻⁶                 B/op
ByteUtilsBenchmark.testSizeOfVarlongSimple:·gc.count                   thrpt    4          ≈ 0               counts
```

Reviewers: Ismael Juma <ismael@juma.me.uk>, Artem Livshits
2022-02-06 13:36:44 -08:00
Justine Olshan e14499948e
KAFKA-13512: Avoid duplicating maps in ZkMetadataCache topic accessors
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Luke Chen <showuon@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2021-12-08 13:57:08 -08:00
Justine Olshan 1c99c0ce15
MINOR: Fix FetchSessionBenchmark (#11501)
Reviewers: David Jacot <djacot@confluent.io>
2021-11-16 10:39:35 +01:00
Justine Olshan e8818e234a
KAFKA-13111: Re-evaluate Fetch Sessions when using topic IDs (#11331)
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>
2021-11-15 10:04:43 +01:00
dengziming 53f5c2606d MINOR: ZkMetadataCache should be in kafka.server.metadata #10956
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>
2021-10-26 14:16:52 -07:00
Justine Olshan b76bcaf3a8
KAFKA-13102: Topic IDs not propagated to metadata cache quickly enough for Fetch path (#11170)
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>
2021-09-24 10:51:08 +02:00
Colin Patrick McCabe 074a3dacca
MINOR: Make ReplicaManager, LogManager, KafkaApis easier to construct (#11320)
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>
2021-09-17 14:12:31 -07:00
José Armando García Sancio 69a4661d7a
KAFKA-13100: Create KRaft controller snapshot during promotion (#11084)
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>
2021-07-20 10:13:01 -07:00
Ismael Juma a46b82bea9
KAFKA-12944: Assume message format version is 3.0 when inter-broker protocol is 3.0 or higher (KIP-724) (#11036)
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>
2021-07-19 05:37:16 -07:00
Justine Olshan 584213ed20
Fix perf regression on LISR requests by asynchronously flushing the partition.metadata file (#11056)
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>
2021-07-15 14:00:32 -07:00
Justine Olshan 2b8aff58b5
KAFKA-10580: Add topic ID support to Fetch request (#9944)
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>
2021-07-07 16:02:37 -07:00
Jeff Kim c671e44b0b
MINOR: Add `KafkaAdminClient.getListOffsetsCalls` benchmark (#10955)
Reviewers: David Jacot <djacot@confluent.io>
2021-07-07 08:58:56 +02:00
Colin Patrick McCabe 7bd55f5156
KAFKA-12998: Implement broker-side KRaft snapshots (#10931)
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>
2021-07-06 16:36:01 -07:00
Colin Patrick McCabe bd668e90c6
MINOR: add MockConfigRepository (#10927)
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>
2021-06-25 16:40:42 -07:00
thomaskwscott bd72ef1bf1
KAFKA-12541; Extend ListOffset to fetch offset with max timestamp (KIP-734) (#10760)
This patch implements KIP-734 as described in https://cwiki.apache.org/confluence/display/KAFKA/KIP-734%3A+Improve+AdminClient.listOffsets+to+return+timestamp+and+offset+for+the+record+with+the+largest+timestamp.

Reviewers: David Jacot <djacot@confluent.io>
2021-06-25 14:29:12 +02:00
Ismael Juma 8b71604c5d
MINOR: Update jmh for async profiler 2.0 support (#10800)
Async profiler 2.0 outputs html5 flame graph files
and supports simultaneous collection of cpu,
allocation and lock profiles in jfr format.

Updated the readme to include an example of the
latter and verified that the Readme commands
work with async profiler 2.0.

Release notes:
* 1.28: https://mail.openjdk.java.net/pipermail/jmh-dev/2021-March/003171.html
* 1.29: https://mail.openjdk.java.net/pipermail/jmh-dev/2021-March/003218.html
* 1.30: https://mail.openjdk.java.net/pipermail/jmh-dev/2021-May/003237.html
* 1.31: https://mail.openjdk.java.net/pipermail/jmh-dev/2021-May/003286.html
* 1.32: https://mail.openjdk.java.net/pipermail/jmh-dev/2021-May/003307.html

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>, Luke Chen <showuon@gmail.com>
2021-06-02 05:55:01 -07:00
Ismael Juma 6b005b2b4e
MINOR: Reduce allocations in requests via buffer caching (#9229)
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>
2021-05-30 12:16:36 -07:00
Ron Dagostino 5b0c58ed53
MINOR: Support using the ZK authorizer with KRaft (#10550)
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>
2021-05-19 10:32:56 -07:00
Colin Patrick McCabe ec38dcb72f
MINOR: support ImplicitLinkedHashCollection#sort (#10456)
Support sorting the elements in ImplicitLinkedHashCollection.
This is useful sometimes in unit tests for comparing collections.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-04-01 18:30:53 -07:00
Justine Olshan 40f001cc53
KAFKA-12426: Missing logic to create partition.metadata files in RaftReplicaManager (#10282)
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>
2021-04-01 15:42:21 -07:00
Ismael Juma 7a3ebbebbc
KAFKA-12415 Prepare for Gradle 7.0 and restrict transitive scope for non api dependencies (#10203)
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>
2021-03-04 11:22:22 -08:00
Chia-Ping Tsai 8205051e90
MINOR: remove FetchResponse.AbortedTransaction and redundant construc… (#9758)
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>
2021-03-04 18:06:50 +08:00
Jason Gustafson 698319b8e2 KAFKA-12278; Ensure exposed api versions are consistent within listener (#10666)
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>
2021-02-18 16:25:51 -08:00
Ron Dagostino a30f92bf59
MINOR: Add KIP-500 BrokerServer and ControllerServer (#10113)
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>
2021-02-17 21:35:13 -08:00
Ismael Juma 744d05b128
KAFKA-12327: Remove MethodHandle usage in CompressionType (#10123)
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>
2021-02-14 08:12:25 -08:00
Justine Olshan 39dcdeffd7
MINOR: Prevent creating partition.metadata until ID can be written (#10041)
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>
2021-02-10 15:22:15 -08:00
Ron Dagostino 42a9355e60
MINOR: Defer log recovery until LogManager startup (#10039)
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>
2021-02-07 15:46:41 -08:00
Boyang Chen d2cb2dc45d
KAFKA-9751: Forward CreateTopicsRequest for FindCoordinator/Metadata when topic creation is needed (#9579)
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>
2021-02-06 13:04:30 -08:00
David Arthur 242f65e5ba
Refactor the MetadataCache into two implementations (#10049)
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>
2021-02-05 16:25:26 -08:00
Ron Dagostino acf39fe94a
MINOR: Allow KafkaApis to be configured for Raft controller quorums (#10045)
`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>
2021-02-05 12:57:44 -08:00
Ron Dagostino c4ea6fb0a7
MINOR: Add ConfigRepository, use in Partition and KafkaApis (#10005)
`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>
2021-02-04 12:58:26 -08:00
Colin Patrick McCabe 772f2cfc82
MINOR: Replace BrokerStates.scala with BrokerState.java (#10028)
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>
2021-02-03 13:41:38 -08:00
Colin Patrick McCabe 1711cfa4eb
KAFKA-12209: Add the timeline data structures for the KIP-631 controller (#9901)
Reviewers: Jun Rao <junrao@gmail.com>
2021-02-02 11:33:55 -08:00
Jason Gustafson 9689a313f5
MINOR: Drop enable.metadata.quorum config (#9934)
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>
2021-01-21 15:16:15 -08:00
Colin Patrick McCabe b5c107363f
KAFKA-12208: Rename AdminManager to ZkAdminManager (#9900)
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>
2021-01-15 12:56:09 -08:00
David Arthur d3f19e4bb0
KAFKA-10825 ZooKeeper ISR manager (#9713)
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
2020-12-21 14:44:02 -05:00
dengziming 5c921afa4a
KAFKA-10547; Add TopicId in MetadataResponse (#9622)
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>
2020-12-18 21:30:52 +00:00
Cheng Tan ae3a6ed990
KAKFA-10619: Idempotent producer will get authorized once it has a WRITE access to at least one topic (KIP-679) (#9485)
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>
2020-12-18 18:08:46 +00:00
David Jacot 02a30a51eb
KAFKA-10740; Replace OffsetsForLeaderEpochRequest.PartitionData with automated protocol (#9689)
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>
2020-12-17 17:40:37 +01:00
Ismael Juma 782175dfbc
MINOR: Simplify ApiKeys by relying on ApiMessageType (#9748)
* 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>
2020-12-16 06:33:10 -08:00
Anastasia Vela 1a10c3445e
KAFKA-10525: Emit JSONs with new auto-generated schema (KIP-673) (#9526)
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>
2020-12-15 14:33:36 +01:00
Ismael Juma 8cabd57612
MINOR: Update jmh to 1.27 for async profiler support (#9129)
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>
2020-12-10 17:56:52 -08:00
Ismael Juma 1f98112e99
MINOR: Remove connection id from Send and consolidate request/message utils (#9714)
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>
2020-12-09 11:15:58 -08:00
Ismael Juma 6f27bb02da
KAFKA-10818: Skip conversion to `Struct` when serializing generated requests/responses (#7409)
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>
2020-12-07 15:39:57 -08:00
David Arthur 633f7cff19
KAFKA-10799 AlterIsr utilizes ReplicaManager ISR metrics (#9677)
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.
2020-12-03 16:11:07 -05:00
David Jacot 10364e4b0c
KAFKA-10739; Replace EpochEndOffset with automated protocol (#9630)
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>
2020-12-03 18:50:29 +01:00
Rajini Sivaram 7ecc3a579a
KAFKA-10554; Perform follower truncation based on diverging epochs in Fetch response (#9382)
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>
2020-12-03 10:12:06 +00:00
Jason Gustafson 6054837c0a
MINOR: Reduce sends created by `SendBuilder` (#9619)
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>
2020-11-19 12:45:23 -08:00
Chia-Ping Tsai 30bc21ca35
KAFKA-9628; Replace Produce request/response with automated protocol (#9401)
This patch rewrites `ProduceRequest` and `ProduceResponse` using the generated protocols. We have also added several new benchmarks to verify no regression in performance. A summary of results is included below:

### Benchmark

1. loop **30** times
1. calculate average

#### kafkatest.benchmarks.core.benchmark_test.Benchmark.test_producer_throughput

> @cluster(num_nodes=5)
> @parametrize(acks=-1, topic=TOPIC_REP_THREE)

- +0.3144915325 %
- 28.08766667 ->  28.1715625 (mb_per_sec)

> @cluster(num_nodes=5)
> @matrix(acks=[1], topic=[TOPIC_REP_THREE], message_size=[100000],compression_type=["none"], security_protocol=['PLAINTEXT'])

- +4.220730323 %
- 157.145 -> 163.7776667 (mb_per_sec)

> @cluster(num_nodes=7)
> @parametrize(acks=1, topic=TOPIC_REP_THREE, num_producers=3)

- +5.996241145%
- 57.64166667 -> 61.098 (mb_per_sec)

> @cluster(num_nodes=5)
> @parametrize(acks=1, topic=TOPIC_REP_THREE)

- +0.3979572536%
- 44.05833333 -> 44.23366667 (mb_per_sec)

> @cluster(num_nodes=5)
> @parametrize(acks=1, topic= TOPIC_REP_ONE)

- +2.228235226%
- 69.23266667 -> 70.77533333 (mb_per_sec)

### JMH results

In short, most ops performance are regression since we have to convert data to protocol data. The cost is inevitable (like other request/response) before we use protocol data directly.

### JMH for ProduceRequest

1. construction regression:
    - 281.474 -> 454.935 ns/op
    - 296.000 -> 1888.000 B/op
1. toErrorResponse regression:
    - 41.942 -> 107.528 ns/op
    - 1216.000 -> 1616.000 B/op
1. toStruct improvement:
    - 255.185 -> 90.728 ns/op
    - 864.000 -> 304.000 B/op

**BEFORE**
```
Benchmark                                                                        Mode  Cnt     Score    Error   Units
ProducerRequestBenchmark.constructorErrorResponse                                avgt   15    41.942 ±  0.036   ns/op
ProducerRequestBenchmark.constructorErrorResponse:·gc.alloc.rate                 avgt   15  6409.263 ±  5.478  MB/sec
ProducerRequestBenchmark.constructorErrorResponse:·gc.alloc.rate.norm            avgt   15   296.000 ±  0.001    B/op
ProducerRequestBenchmark.constructorErrorResponse:·gc.churn.G1_Eden_Space        avgt   15  6416.420 ± 76.071  MB/sec
ProducerRequestBenchmark.constructorErrorResponse:·gc.churn.G1_Eden_Space.norm   avgt   15   296.331 ±  3.539    B/op
ProducerRequestBenchmark.constructorErrorResponse:·gc.churn.G1_Old_Gen           avgt   15     0.002 ±  0.002  MB/sec
ProducerRequestBenchmark.constructorErrorResponse:·gc.churn.G1_Old_Gen.norm      avgt   15    ≈ 10⁻⁴             B/op
ProducerRequestBenchmark.constructorErrorResponse:·gc.count                      avgt   15   698.000           counts
ProducerRequestBenchmark.constructorErrorResponse:·gc.time                       avgt   15   378.000               ms
ProducerRequestBenchmark.constructorProduceRequest                               avgt   15   281.474 ±  3.286   ns/op
ProducerRequestBenchmark.constructorProduceRequest:·gc.alloc.rate                avgt   15  3923.868 ± 46.303  MB/sec
ProducerRequestBenchmark.constructorProduceRequest:·gc.alloc.rate.norm           avgt   15  1216.000 ±  0.001    B/op
ProducerRequestBenchmark.constructorProduceRequest:·gc.churn.G1_Eden_Space       avgt   15  3923.375 ± 59.568  MB/sec
ProducerRequestBenchmark.constructorProduceRequest:·gc.churn.G1_Eden_Space.norm  avgt   15  1215.844 ± 11.184    B/op
ProducerRequestBenchmark.constructorProduceRequest:·gc.churn.G1_Old_Gen          avgt   15     0.004 ±  0.001  MB/sec
ProducerRequestBenchmark.constructorProduceRequest:·gc.churn.G1_Old_Gen.norm     avgt   15     0.001 ±  0.001    B/op
ProducerRequestBenchmark.constructorProduceRequest:·gc.count                     avgt   15   515.000           counts
ProducerRequestBenchmark.constructorProduceRequest:·gc.time                      avgt   15   279.000               ms
ProducerRequestBenchmark.constructorStruct                                       avgt   15   255.185 ±  0.069   ns/op
ProducerRequestBenchmark.constructorStruct:·gc.alloc.rate                        avgt   15  3074.889 ±  0.823  MB/sec
ProducerRequestBenchmark.constructorStruct:·gc.alloc.rate.norm                   avgt   15   864.000 ±  0.001    B/op
ProducerRequestBenchmark.constructorStruct:·gc.churn.G1_Eden_Space               avgt   15  3077.737 ± 31.537  MB/sec
ProducerRequestBenchmark.constructorStruct:·gc.churn.G1_Eden_Space.norm          avgt   15   864.800 ±  8.823    B/op
ProducerRequestBenchmark.constructorStruct:·gc.churn.G1_Old_Gen                  avgt   15     0.003 ±  0.001  MB/sec
ProducerRequestBenchmark.constructorStruct:·gc.churn.G1_Old_Gen.norm             avgt   15     0.001 ±  0.001    B/op
ProducerRequestBenchmark.constructorStruct:·gc.count                             avgt   15   404.000           counts
ProducerRequestBenchmark.constructorStruct:·gc.time                              avgt   15   214.000               ms
```

**AFTER**
```
Benchmark                                                                        Mode  Cnt     Score    Error   Units
ProducerRequestBenchmark.constructorErrorResponse                                avgt   15   107.528 ±  0.270   ns/op
ProducerRequestBenchmark.constructorErrorResponse:·gc.alloc.rate                 avgt   15  4864.899 ± 12.132  MB/sec
ProducerRequestBenchmark.constructorErrorResponse:·gc.alloc.rate.norm            avgt   15   576.000 ±  0.001    B/op
ProducerRequestBenchmark.constructorErrorResponse:·gc.churn.G1_Eden_Space        avgt   15  4868.023 ± 61.943  MB/sec
ProducerRequestBenchmark.constructorErrorResponse:·gc.churn.G1_Eden_Space.norm   avgt   15   576.371 ±  7.331    B/op
ProducerRequestBenchmark.constructorErrorResponse:·gc.churn.G1_Old_Gen           avgt   15     0.005 ±  0.001  MB/sec
ProducerRequestBenchmark.constructorErrorResponse:·gc.churn.G1_Old_Gen.norm      avgt   15     0.001 ±  0.001    B/op
ProducerRequestBenchmark.constructorErrorResponse:·gc.count                      avgt   15   639.000           counts
ProducerRequestBenchmark.constructorErrorResponse:·gc.time                       avgt   15   339.000               ms
ProducerRequestBenchmark.constructorProduceRequest                               avgt   15   454.935 ±  0.332   ns/op
ProducerRequestBenchmark.constructorProduceRequest:·gc.alloc.rate                avgt   15  3769.014 ±  2.767  MB/sec
ProducerRequestBenchmark.constructorProduceRequest:·gc.alloc.rate.norm           avgt   15  1888.000 ±  0.001    B/op
ProducerRequestBenchmark.constructorProduceRequest:·gc.churn.G1_Eden_Space       avgt   15  3763.407 ± 31.530  MB/sec
ProducerRequestBenchmark.constructorProduceRequest:·gc.churn.G1_Eden_Space.norm  avgt   15  1885.190 ± 15.594    B/op
ProducerRequestBenchmark.constructorProduceRequest:·gc.churn.G1_Old_Gen          avgt   15     0.004 ±  0.001  MB/sec
ProducerRequestBenchmark.constructorProduceRequest:·gc.churn.G1_Old_Gen.norm     avgt   15     0.002 ±  0.001    B/op
ProducerRequestBenchmark.constructorProduceRequest:·gc.count                     avgt   15   494.000           counts
ProducerRequestBenchmark.constructorProduceRequest:·gc.time                      avgt   15   264.000               ms
ProducerRequestBenchmark.constructorStruct                                       avgt   15    90.728 ±  0.695   ns/op
ProducerRequestBenchmark.constructorStruct:·gc.alloc.rate                        avgt   15  3043.140 ± 23.246  MB/sec
ProducerRequestBenchmark.constructorStruct:·gc.alloc.rate.norm                   avgt   15   304.000 ±  0.001    B/op
ProducerRequestBenchmark.constructorStruct:·gc.churn.G1_Eden_Space               avgt   15  3047.251 ± 59.638  MB/sec
ProducerRequestBenchmark.constructorStruct:·gc.churn.G1_Eden_Space.norm          avgt   15   304.404 ±  5.034    B/op
ProducerRequestBenchmark.constructorStruct:·gc.churn.G1_Old_Gen                  avgt   15     0.003 ±  0.001  MB/sec
ProducerRequestBenchmark.constructorStruct:·gc.churn.G1_Old_Gen.norm             avgt   15    ≈ 10⁻⁴             B/op
ProducerRequestBenchmark.constructorStruct:·gc.count                             avgt   15   400.000           counts
ProducerRequestBenchmark.constructorStruct:·gc.time                              avgt   15   205.000               ms
```
### JMH for ProduceResponse

1. construction regression:
    - 3.293 -> 303.226 ns/op
    - 24.000 -> 1848.000 B/op
1. toStruct improvement:
    - 825.889 -> 311.725 ns/op
    - 2208.000 -> 896.000 B/op

**BEFORE**

```
Benchmark                                                                          Mode  Cnt     Score    Error   Units
ProducerResponseBenchmark.constructorProduceResponse                               avgt   15     3.293 ±  0.004   ns/op
ProducerResponseBenchmark.constructorProduceResponse:·gc.alloc.rate                avgt   15  6619.731 ±  9.075  MB/sec
ProducerResponseBenchmark.constructorProduceResponse:·gc.alloc.rate.norm           avgt   15    24.000 ±  0.001    B/op
ProducerResponseBenchmark.constructorProduceResponse:·gc.churn.G1_Eden_Space       avgt   15  6618.648 ±  0.153  MB/sec
ProducerResponseBenchmark.constructorProduceResponse:·gc.churn.G1_Eden_Space.norm  avgt   15    23.996 ±  0.033    B/op
ProducerResponseBenchmark.constructorProduceResponse:·gc.churn.G1_Old_Gen          avgt   15     0.003 ±  0.002  MB/sec
ProducerResponseBenchmark.constructorProduceResponse:·gc.churn.G1_Old_Gen.norm     avgt   15    ≈ 10⁻⁵             B/op
ProducerResponseBenchmark.constructorProduceResponse:·gc.count                     avgt   15   720.000           counts
ProducerResponseBenchmark.constructorProduceResponse:·gc.time                      avgt   15   383.000               ms
ProducerResponseBenchmark.constructorStruct                                        avgt   15   825.889 ±  0.638   ns/op
ProducerResponseBenchmark.constructorStruct:·gc.alloc.rate                         avgt   15  2428.000 ±  1.899  MB/sec
ProducerResponseBenchmark.constructorStruct:·gc.alloc.rate.norm                    avgt   15  2208.000 ±  0.001    B/op
ProducerResponseBenchmark.constructorStruct:·gc.churn.G1_Eden_Space                avgt   15  2430.196 ± 55.894  MB/sec
ProducerResponseBenchmark.constructorStruct:·gc.churn.G1_Eden_Space.norm           avgt   15  2210.001 ± 51.009    B/op
ProducerResponseBenchmark.constructorStruct:·gc.churn.G1_Old_Gen                   avgt   15     0.003 ±  0.001  MB/sec
ProducerResponseBenchmark.constructorStruct:·gc.churn.G1_Old_Gen.norm              avgt   15     0.002 ±  0.001    B/op
ProducerResponseBenchmark.constructorStruct:·gc.count                              avgt   15   319.000           counts
ProducerResponseBenchmark.constructorStruct:·gc.time                               avgt   15   166.000               ms
```

**AFTER**

```
Benchmark                                                                          Mode  Cnt     Score    Error   Units
ProducerResponseBenchmark.constructorProduceResponse                               avgt   15   303.226 ±  0.517   ns/op
ProducerResponseBenchmark.constructorProduceResponse:·gc.alloc.rate                avgt   15  5534.940 ±  9.439  MB/sec
ProducerResponseBenchmark.constructorProduceResponse:·gc.alloc.rate.norm           avgt   15  1848.000 ±  0.001    B/op
ProducerResponseBenchmark.constructorProduceResponse:·gc.churn.G1_Eden_Space       avgt   15  5534.046 ± 51.849  MB/sec
ProducerResponseBenchmark.constructorProduceResponse:·gc.churn.G1_Eden_Space.norm  avgt   15  1847.710 ± 18.105    B/op
ProducerResponseBenchmark.constructorProduceResponse:·gc.churn.G1_Old_Gen          avgt   15     0.007 ±  0.001  MB/sec
ProducerResponseBenchmark.constructorProduceResponse:·gc.churn.G1_Old_Gen.norm     avgt   15     0.002 ±  0.001    B/op
ProducerResponseBenchmark.constructorProduceResponse:·gc.count                     avgt   15   602.000           counts
ProducerResponseBenchmark.constructorProduceResponse:·gc.time                      avgt   15   318.000               ms
ProducerResponseBenchmark.constructorStruct                                        avgt   15   311.725 ±  3.132   ns/op
ProducerResponseBenchmark.constructorStruct:·gc.alloc.rate                         avgt   15  2610.602 ± 25.964  MB/sec
ProducerResponseBenchmark.constructorStruct:·gc.alloc.rate.norm                    avgt   15   896.000 ±  0.001    B/op
ProducerResponseBenchmark.constructorStruct:·gc.churn.G1_Eden_Space                avgt   15  2613.021 ± 42.965  MB/sec
ProducerResponseBenchmark.constructorStruct:·gc.churn.G1_Eden_Space.norm           avgt   15   896.824 ± 11.331    B/op
ProducerResponseBenchmark.constructorStruct:·gc.churn.G1_Old_Gen                   avgt   15     0.003 ±  0.001  MB/sec
ProducerResponseBenchmark.constructorStruct:·gc.churn.G1_Old_Gen.norm              avgt   15     0.001 ±  0.001    B/op
ProducerResponseBenchmark.constructorStruct:·gc.count                              avgt   15   343.000           counts
ProducerResponseBenchmark.constructorStruct:·gc.time                               avgt   15   194.000               ms
```

Reviewers: David Jacot <djacot@confluent.io>, Jason Gustafson <jason@confluent.io>
2020-11-18 13:44:21 -08:00
Boyang Chen bb34c5c8cc
KAFKA-10350: add forwarding manager implementation with metrics (#9580)
add forwarding manager implementation with metrics

Reviewers: David Arthur <mumrah@gmail.com>
2020-11-11 23:21:10 -08:00
Boyang Chen 0814e4f645
KAFKA-10181: Use Envelope RPC to do redirection for (Incremental)AlterConfig, AlterClientQuota and CreateTopics (#9103)
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>
2020-11-04 14:21:44 -08:00
Boyang Chen 9f26906fcc
Revert "KAFKA-9705 part 1: add KIP-590 request header fields (#9144)" (#9523)
This reverts commit 21dc5231ce as we decide to use Envelope for redirection instead of initial principal.

Reviewers: Jason Gustafson <jason@confluent.io>
2020-10-28 22:57:10 -07:00
Kowshik Prakasam fb4f297207
KAFKA-10028: Implement write path for feature versioning system (KIP-584) (#9001)
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>
2020-10-07 10:23:16 -07:00
David Arthur 57de67db22
KAFKA-8836; Add `AlterISR` RPC and use it for ISR modifications (#9100)
This patch implements [KIP-497](https://cwiki.apache.org/confluence/display/KAFKA/KIP-497%3A+Add+inter-broker+API+to+alter+ISR), which introduces an asynchronous API for partition leaders to update ISR state.

Reviewers: Jason Gustafson <jason@confluent.io>
2020-09-24 16:28:25 -07:00
Chia-Ping Tsai 4b6d8da9fd
KAFKA-10438: Lazy initialization of record header to reduce memory usage (#9223)
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>
2020-09-21 08:03:49 -07:00
David Arthur 1a9697430a
KAFKA-8806 Reduce calls to validateOffsetsIfNeeded (#7222)
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.
2020-08-21 10:25:52 -04:00
Boyang Chen 21dc5231ce
KAFKA-9705 part 1: add KIP-590 request header fields (#9144)
Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Jacot <david.jacot@gmail.com>
2020-08-18 13:38:59 -07:00
David Arthur 4cd2396db3
KAFKA-9629 Use generated protocol for Fetch API (#9008)
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>
2020-07-30 13:29:39 -04:00
Brian Byrne 99472c54f0
KAFKA-10158: Fix flaky testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress (#9022)
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>
2020-07-26 08:28:47 -07:00
David Jacot a5ffd1ca44
KAFKA-10163; Throttle Create Topic, Create Partition and Delete Topic Operations (KIP-599, Part I, Broker Changes) (#8933)
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>
2020-07-22 16:38:55 +01:00
Ismael Juma e2e2c628b9
KAFKA-10074: Improve performance of `matchingAcls` (#8769)
This PR reduces allocations by using a plain old `foreach` in
`matchingAcls` and improves `AclSeqs.find` to only search the inner
collections that are required to find a match (instead of searching all
of them).

A recent change (90bbeedf52) in `matchingAcls` to remove `filterKeys` in
favor of filtering inside `flatMap` caused a performance regression in
cases where there are large number of topics, prefix ACLs and
TreeMap.from/to filtering is ineffective. In such cases, we rely on
string comparisons to exclude entries from the ACL cache that are not
relevant.

This issue is not present in any release yet, so we should include the
simple fix in the 2.6 branch.

The original benchmark did not show a performance difference, so I
adjusted the benchmark to stress the relevant code more. More
specifically, `aclCacheSnapshot.from(...).to(...)` returns nearly 20000
entries where each map value contains 1000 AclEntries. Out of the 200k
AclEntries, only 1050 are retained due to the `startsWith` filtering.

This is the case where the implementation in master is least
efficient when compared to the previous version and the version in this
PR.

The adjusted benchmark results for testAuthorizer are 4.532ms for
master, 2.903ms for the previous version and 2.877ms for this PR.
Normalized allocation rate was 593 KB/op for master, 597 KB/op for the
previous version and 101 KB/s for this PR. Full results follow:

master with adjusted benchmark:
```
Benchmark                                                                 (aclCount)  (resourceCount)  Mode  Cnt          Score          Error   Units
AclAuthorizerBenchmark.testAclsIterator                                           50           200000  avgt    5        680.805 ±       44.318   ms/op
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate                            50           200000  avgt    5        549.879 ±       36.259  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate.norm                       50           200000  avgt    5  411457042.000 ±     4805.461    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space                   50           200000  avgt    5        331.110 ±       95.821  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space.norm              50           200000  avgt    5  247799480.320 ± 72877192.319    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space               50           200000  avgt    5          0.891 ±        3.183  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space.norm          50           200000  avgt    5     667593.387 ±  2369888.357    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.count                                 50           200000  avgt    5         28.000                 counts
AclAuthorizerBenchmark.testAclsIterator:·gc.time                                  50           200000  avgt    5       3458.000                     ms
AclAuthorizerBenchmark.testAuthorizer                                             50           200000  avgt    5          4.532 ±        0.546   ms/op
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate                              50           200000  avgt    5        119.036 ±       14.261  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate.norm                         50           200000  avgt    5     593524.310 ±       22.452    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Eden_Space                     50           200000  avgt    5        117.091 ±     1008.188  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Eden_Space.norm                50           200000  avgt    5     598574.303 ±  5153905.271    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Survivor_Space                 50           200000  avgt    5          0.034 ±        0.291  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Survivor_Space.norm            50           200000  avgt    5        173.001 ±     1489.593    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.count                                   50           200000  avgt    5          1.000                 counts
AclAuthorizerBenchmark.testAuthorizer:·gc.time                                    50           200000  avgt    5         13.000                     ms
```

master with filterKeys like 90bbeedf52 and adjusted benchmark:
```
Benchmark                                                                 (aclCount)  (resourceCount)  Mode  Cnt          Score          Error   Units
AclAuthorizerBenchmark.testAclsIterator                                           50           200000  avgt    5        729.163 ±       20.842   ms/op
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate                            50           200000  avgt    5        513.005 ±       13.966  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate.norm                       50           200000  avgt    5  411459778.400 ±     3178.045    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space                   50           200000  avgt    5        307.041 ±       94.544  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space.norm              50           200000  avgt    5  246385400.686 ± 82294899.881    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space               50           200000  avgt    5          1.571 ±        2.590  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space.norm          50           200000  avgt    5    1258291.200 ±  2063669.849    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.count                                 50           200000  avgt    5         33.000                 counts
AclAuthorizerBenchmark.testAclsIterator:·gc.time                                  50           200000  avgt    5       3266.000                     ms
AclAuthorizerBenchmark.testAuthorizer                                             50           200000  avgt    5          2.903 ±        0.175   ms/op
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate                              50           200000  avgt    5        187.088 ±       11.301  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate.norm                         50           200000  avgt    5     597962.743 ±       14.237    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Eden_Space                     50           200000  avgt    5        118.602 ±     1021.202  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.churn.G1_Eden_Space.norm                50           200000  avgt    5     383359.632 ±  3300842.044    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.count                                   50           200000  avgt    5          1.000                 counts
AclAuthorizerBenchmark.testAuthorizer:·gc.time                                    50           200000  avgt    5         14.000                     ms
```

This PR with adjusted benchmark:
```
Benchmark                                                                 (aclCount)  (resourceCount)  Mode  Cnt          Score          Error   Units
AclAuthorizerBenchmark.testAclsIterator                                           50           200000  avgt    5        706.774 ±       32.353   ms/op
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate                            50           200000  avgt    5        529.879 ±       25.416  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.alloc.rate.norm                       50           200000  avgt    5  411458751.497 ±     4424.187    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space                   50           200000  avgt    5        310.559 ±      112.310  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Eden_Space.norm              50           200000  avgt    5  241364219.611 ± 97317733.967    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Old_Gen                      50           200000  avgt    5          0.690 ±        5.937  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Old_Gen.norm                 50           200000  avgt    5     531278.507 ±  4574468.166    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space               50           200000  avgt    5          2.550 ±       17.243  MB/sec
AclAuthorizerBenchmark.testAclsIterator:·gc.churn.G1_Survivor_Space.norm          50           200000  avgt    5    1969325.592 ± 13278191.648    B/op
AclAuthorizerBenchmark.testAclsIterator:·gc.count                                 50           200000  avgt    5         32.000                 counts
AclAuthorizerBenchmark.testAclsIterator:·gc.time                                  50           200000  avgt    5       3489.000                     ms
AclAuthorizerBenchmark.testAuthorizer                                             50           200000  avgt    5          2.877 ±        0.530   ms/op
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate                              50           200000  avgt    5         31.963 ±        5.912  MB/sec
AclAuthorizerBenchmark.testAuthorizer:·gc.alloc.rate.norm                         50           200000  avgt    5     101057.225 ±        9.468    B/op
AclAuthorizerBenchmark.testAuthorizer:·gc.count                                   50           200000  avgt    5            ≈ 0                 counts
```

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2020-06-01 07:01:18 -07:00
Jason Gustafson 2cbf6be54a
KAFKA-9952; Remove immediate fetch completion logic on high watermark updates (#8709)
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>
2020-05-27 14:41:49 -07:00
Lucas Bradstreet cfc34cace5
MINOR: reduce allocations in log start and recovery checkpoints (#8467)
For brokers with replica counts > 4000, allocations from logsByDir become
substantial. logsByDir is called often by LogManager.checkpointLogRecoveryOffsets
and LogManager.checkpointLogStartOffsets. The approach used is similar to the
one from the checkpointHighwatermarks change in
https://github.com/apache/kafka/pull/6741.

Are there better ways to structure out data structure to avoid creating logsByDir on
demand for each checkpoint iteration? This micro-optimization will help as is, but if
we can avoid doing this completely it'd be better.

JMH benchmark results:
```
Before:
Benchmark                                                                      (numPartitions)  (numTopics)   Mode  Cnt        Score        Error   Units
CheckpointBench.measureCheckpointLogStartOffsets                                             3          100  thrpt   15        2.233 ±      0.013  ops/ms
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate                              3          100  thrpt   15      477.097 ±     49.731  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate.norm                         3          100  thrpt   15   246083.007 ±     33.052    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space                     3          100  thrpt   15      475.683 ±     55.569  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space.norm                3          100  thrpt   15   245474.040 ±  14968.328    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen                        3          100  thrpt   15        0.001 ±      0.001  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen.norm                   3          100  thrpt   15        0.341 ±      0.268    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.count                                   3          100  thrpt   15      129.000               counts
CheckpointBench.measureCheckpointLogStartOffsets:·gc.time                                    3          100  thrpt   15       52.000                   ms
CheckpointBench.measureCheckpointLogStartOffsets                                             3         1000  thrpt   15        0.572 ±      0.004  ops/ms
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate                              3         1000  thrpt   15     1360.240 ±    150.539  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate.norm                         3         1000  thrpt   15  2750221.257 ±    891.024    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space                     3         1000  thrpt   15     1362.908 ±    148.799  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space.norm                3         1000  thrpt   15  2756395.092 ±  44671.843    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen                        3         1000  thrpt   15        0.017 ±      0.008  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen.norm                   3         1000  thrpt   15       33.611 ±     14.401    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.count                                   3         1000  thrpt   15      273.000               counts
CheckpointBench.measureCheckpointLogStartOffsets:·gc.time                                    3         1000  thrpt   15      186.000                   ms
CheckpointBench.measureCheckpointLogStartOffsets                                             3         2000  thrpt   15        0.266 ±      0.002  ops/ms
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate                              3         2000  thrpt   15     1342.557 ±    171.260  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate.norm                         3         2000  thrpt   15  5877881.729 ±   3695.086    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space                     3         2000  thrpt   15     1343.965 ±    186.069  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space.norm                3         2000  thrpt   15  5877788.561 ± 168540.343    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen                        3         2000  thrpt   15        0.081 ±      0.043  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen.norm                   3         2000  thrpt   15      351.277 ±    167.006    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.count                                   3         2000  thrpt   15      253.000               counts
CheckpointBench.measureCheckpointLogStartOffsets:·gc.time                                    3         2000  thrpt   15      231.000                   ms
JMH benchmarks done

After:
CheckpointBench.measureCheckpointLogStartOffsets                                             3          100  thrpt   15        2.809 ±     0.129  ops/ms
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate                              3          100  thrpt   15      211.248 ±    25.953  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate.norm                         3          100  thrpt   15    86533.838 ±  3763.989    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space                     3          100  thrpt   15      211.512 ±    38.669  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space.norm                3          100  thrpt   15    86228.552 ±  9590.781    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen                        3          100  thrpt   15       ≈ 10⁻³              MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen.norm                   3          100  thrpt   15        0.140 ±     0.111    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.count                                   3          100  thrpt   15       57.000              counts
CheckpointBench.measureCheckpointLogStartOffsets:·gc.time                                    3          100  thrpt   15       25.000                  ms
CheckpointBench.measureCheckpointLogStartOffsets                                             3         1000  thrpt   15        1.046 ±     0.030  ops/ms
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate                              3         1000  thrpt   15      524.597 ±    74.793  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate.norm                         3         1000  thrpt   15   582898.889 ± 37552.262    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space                     3         1000  thrpt   15      519.675 ±    89.754  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space.norm                3         1000  thrpt   15   576371.150 ± 55972.955    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen                        3         1000  thrpt   15        0.009 ±     0.005  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen.norm                   3         1000  thrpt   15        9.920 ±     5.375    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.count                                   3         1000  thrpt   15      111.000              counts
CheckpointBench.measureCheckpointLogStartOffsets:·gc.time                                    3         1000  thrpt   15       56.000                  ms
CheckpointBench.measureCheckpointLogStartOffsets                                             3         2000  thrpt   15        0.617 ±     0.007  ops/ms
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate                              3         2000  thrpt   15      573.061 ±    95.931  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.alloc.rate.norm                         3         2000  thrpt   15  1092098.004 ± 75140.633    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space                     3         2000  thrpt   15      572.448 ±    97.960  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Eden_Space.norm                3         2000  thrpt   15  1091290.460 ± 85946.164    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen                        3         2000  thrpt   15        0.010 ±     0.012  MB/sec
CheckpointBench.measureCheckpointLogStartOffsets:·gc.churn.G1_Old_Gen.norm                   3         2000  thrpt   15       19.990 ±    24.407    B/op
CheckpointBench.measureCheckpointLogStartOffsets:·gc.count                                   3         2000  thrpt   15      109.000              counts
CheckpointBench.measureCheckpointLogStartOffsets:·gc.time                                    3         2000  thrpt   15       67.000                  ms
JMH benchmarks done

```

For the 2000 topic, 3 partition case, we see a reduction in normalized allocations from 5877881B/op to 1284190.774B/op, a reduction of 78%.

Some allocation profiles from a mid sized broker follow. I have seen worse, but these
add up to around 3.8% on a broker that saw GC overhead in CPU time of around 30%.
You could argue that this is relatively small, but it seems worthwhile for a low risk change.

![image](https://user-images.githubusercontent.com/252189/79058104-33e91d80-7c1e-11ea-99c9-0cf2e3571e1f.png)
![image](https://user-images.githubusercontent.com/252189/79058105-38add180-7c1e-11ea-8bfd-6e6eafb0c794.png)

Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-04-25 13:54:59 -07:00
Lucas Bradstreet 37990f9099
MINOR: fix inaccurate RecordBatchIterationBenchmark.measureValidation benchmark (#8428)
KAFKA-9820 (https://github.com/apache/kafka/pull/8422) added a benchmark of LogValidator.validateMessagesAndAssignOffsetsCompressed. Unfortunately it instantiated BrokerTopicStats within the benchmark itself, and it is expensive. The fixed benchmark does not change the outcome of the improvement in KAFKA-9820, and actually increases the magnitude of the improvement in percentage terms.

```
Updated benchmark before KAFKA-9820:
Benchmark                                                                     (bufferSupplierStr)  (bytes)  (compressionType)  (maxBatchSize)  (messageSize)  (messageVersion)   Mode  Cnt       Score      Error   Units
RecordBatchIterationBenchmark.measureValidation                                        NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15  164173.236 ± 2927.701   ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate                         NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15   20440.980 ±  364.411  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm                    NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15  137120.002 ±    0.002    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space                NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15   20708.378 ±  372.041  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space.norm           NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15  138913.935 ±  398.960    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen                   NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15       0.547 ±    0.107  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen.norm              NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15       3.664 ±    0.689    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.count                              NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15    2713.000             counts
RecordBatchIterationBenchmark.measureValidation:·gc.time                               NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15    1398.000                 ms
RecordBatchIterationBenchmark.measureValidation                                        NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15  164305.533 ± 5143.457   ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate                         NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15   20490.828 ±  641.408  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm                    NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15  137328.002 ±    0.002    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space                NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15   20767.922 ±  648.843  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space.norm           NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15  139185.616 ±  325.790    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen                   NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15       0.681 ±    0.053  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen.norm              NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15       4.560 ±    0.292    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.count                              NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15    3101.000             counts
RecordBatchIterationBenchmark.measureValidation:·gc.time                               NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15    1538.000                 ms
RecordBatchIterationBenchmark.measureValidation                                        NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15  169572.635 ±  595.613   ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate                         NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15   21129.934 ±   74.618  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm                    NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15  137216.002 ±    0.002    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space                NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15   21410.416 ±   70.458  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space.norm           NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15  139037.806 ±  309.278    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen                   NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15       0.312 ±    0.420  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen.norm              NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15       2.026 ±    2.725    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.count                              NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15    3398.000             counts
RecordBatchIterationBenchmark.measureValidation:·gc.time                               NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15    1701.000                 ms
JMH benchmarks done


Updated benchmark after KAFKA-9820:
Benchmark                                                                     (bufferSupplierStr)  (bytes)  (compressionType)  (maxBatchSize)  (messageSize)  (messageVersion)   Mode  Cnt       Score     Error   Units
RecordBatchIterationBenchmark.measureValidation                                        NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15  322678.586 ± 254.126   ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate                         NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15   20376.474 ±  15.326  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm                    NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15   69544.001 ±   0.001    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space                NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15   20485.394 ±  44.087  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space.norm           NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15   69915.744 ± 143.372    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen                   NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15       0.027 ±   0.002  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen.norm              NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15       0.091 ±   0.008    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.count                              NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15    3652.000            counts
RecordBatchIterationBenchmark.measureValidation:·gc.time                               NO_CACHING   RANDOM                LZ4               1           1000                 2  thrpt   15    1773.000                ms
RecordBatchIterationBenchmark.measureValidation                                        NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15  321332.070 ± 869.841   ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate                         NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15   20303.259 ±  55.609  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm                    NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15   69600.001 ±   0.001    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space                NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15   20394.052 ±  72.842  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space.norm           NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15   69911.238 ± 160.177    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen                   NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15       0.028 ±   0.003  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen.norm              NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15       0.096 ±   0.010    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.count                              NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15    3637.000            counts
RecordBatchIterationBenchmark.measureValidation:·gc.time                               NO_CACHING   RANDOM                LZ4               2           1000                 2  thrpt   15    1790.000                ms
RecordBatchIterationBenchmark.measureValidation                                        NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15  315490.355 ± 271.921   ops/s
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate                         NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15   19943.166 ±  21.235  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm                    NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15   69640.001 ±   0.001    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space                NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15   20020.263 ±  43.144  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Eden_Space.norm           NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15   69909.228 ± 136.413    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen                   NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15       0.026 ±   0.002  MB/sec
RecordBatchIterationBenchmark.measureValidation:·gc.churn.G1_Old_Gen.norm              NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15       0.090 ±   0.008    B/op
RecordBatchIterationBenchmark.measureValidation:·gc.count                              NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15    3571.000            counts
RecordBatchIterationBenchmark.measureValidation:·gc.time                               NO_CACHING   RANDOM                LZ4              10           1000                 2  thrpt   15    1764.000                ms
```

Reviewers: Ismael Juma <ismael@juma.me.uk>
2020-04-05 14:24:54 -07:00
Lucas Bradstreet 46540eb5e0
KAFKA-9820: validateMessagesAndAssignOffsetsCompressed allocates unused iterator (#8422)
3e9d1c1411 introduced skipKeyValueIterator(s) which were intended to be used, but in this case were created but were not used in offset validation.

A subset of the benchmark results follow. Looks like a 20% improvement in validation performance and a 40% reduction in garbage allocation for 1-2 batch sizes.

**# Parameters: (bufferSupplierStr = NO_CACHING, bytes = RANDOM, compressionType = LZ4, maxBatchSize = 1, messageSize = 1000, messageVersion = 2)**

Before:
Result "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation":
  64851.837 ±(99.9%) 944.248 ops/s [Average]              
  (min, avg, max) = (64505.317, 64851.837, 65114.359), stdev = 245.218
  CI (99.9%): [63907.589, 65796.084] (assumes normal distribution)                                       
                                                             
"org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm":
  164088.003 ±(99.9%) 0.004 B/op [Average]                                                                                 
  (min, avg, max) = (164088.001, 164088.003, 164088.004), stdev = 0.001
  CI (99.9%): [164087.998, 164088.007] (assumes normal distribution)

After:

Result "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation":                                      
  78910.273 ±(99.9%) 707.024 ops/s [Average]                                                                               
  (min, avg, max) = (78785.486, 78910.273, 79234.007), stdev = 183.612                                                     
  CI (99.9%): [78203.249, 79617.297] (assumes normal distribution)                                       

"org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm":                                                                                                                                   
  96440.002 ±(99.9%) 0.001 B/op [Average]                                                                                  
  (min, avg, max) = (96440.002, 96440.002, 96440.002), stdev = 0.001                                                       
  CI (99.9%): [96440.002, 96440.003] (assumes normal distribution)   

 **# Parameters: (bufferSupplierStr = NO_CACHING, bytes = RANDOM, compressionType = LZ4, maxBatchSize = 2, messageSize = 1000, messageVersion = 2)**

Before:
Result "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation":                                      
  64815.364 ±(99.9%) 639.309 ops/s [Average]                                                                               
  (min, avg, max) = (64594.545, 64815.364, 64983.305), stdev = 166.026                                                                                                                                                                                
  CI (99.9%): [64176.056, 65454.673] (assumes normal distribution)                                                         
                                                                                                                                                                                        "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm":        
  163944.003 ±(99.9%) 0.001 B/op [Average]                                                                                 
  (min, avg, max) = (163944.002, 163944.003, 163944.003), stdev = 0.001                                                    
  CI (99.9%): [163944.002, 163944.004] (assumes normal distribution)                                     

After:
Result "org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation":
  77075.096 ±(99.9%) 201.092 ops/s [Average]              
  (min, avg, max) = (77021.537, 77075.096, 77129.693), stdev = 52.223
  CI (99.9%): [76874.003, 77276.188] (assumes normal distribution)                                       
                                                             
"org.apache.kafka.jmh.record.RecordBatchIterationBenchmark.measureValidation:·gc.alloc.rate.norm":
  96504.002 ±(99.9%) 0.003 B/op [Average]                                                                                  
  (min, avg, max) = (96504.001, 96504.002, 96504.003), stdev = 0.001
  CI (99.9%): [96503.999, 96504.005] (assumes normal distribution)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2020-04-04 10:05:51 -07:00
Ismael Juma 222726d6f9
KAFKA-9373: Reduce shutdown time by avoiding unnecessary loading of indexes (#8346)
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>
2020-03-26 04:26:51 -07:00
Gardner Vickers 8cf781ef01
MINOR: Improve performance of checkpointHighWatermarks, patch 1/2 (#6741)
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>
2020-03-25 20:53:42 -07:00
Manikumar Reddy a0e1407820
KAFKA-9670; Reduce allocations in Metadata Response preparation (#8236)
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>
2020-03-16 09:30:48 -07:00
jiao e3ccf20794 KAFKA-9685: Solve Set concatenation perf issue in AclAuthorizer
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
2020-03-13 20:53:29 +05:30
Manikumar Reddy 8dff0b168a Kafka 9626: Improve ACLAuthorizer.acls() performance
This PR avoids creation of unnecessary sets in AclAuthorizer.acls() method implementation.

Perf results:
**Old**
```
Benchmark                                (aclCount)  (resourceCount)  Mode  Cnt    Score   Error  Units
AclAuthorizerBenchmark.testAclsIterator           5             5000  avgt   15    5.821 ? 0.309  ms/op
AclAuthorizerBenchmark.testAclsIterator           5            10000  avgt   15   15.303 ? 0.107  ms/op
AclAuthorizerBenchmark.testAclsIterator           5            50000  avgt   15   74.976 ? 0.543  ms/op
AclAuthorizerBenchmark.testAclsIterator          10             5000  avgt   15   15.366 ? 0.184  ms/op
AclAuthorizerBenchmark.testAclsIterator          10            10000  avgt   15   29.899 ? 0.129  ms/op
AclAuthorizerBenchmark.testAclsIterator          10            50000  avgt   15  167.301 ? 1.723  ms/op
AclAuthorizerBenchmark.testAclsIterator          15             5000  avgt   15   21.980 ? 0.114  ms/op
AclAuthorizerBenchmark.testAclsIterator          15            10000  avgt   15   44.385 ? 0.255  ms/op
AclAuthorizerBenchmark.testAclsIterator          15            50000  avgt   15  241.919 ? 3.955  ms/op
```
**New**

```
Benchmark                                (aclCount)  (resourceCount)  Mode  Cnt   Score   Error  Units
AclAuthorizerBenchmark.testAclsIterator           5             5000  avgt   15   0.666 ? 0.004  ms/op
AclAuthorizerBenchmark.testAclsIterator           5            10000  avgt   15   1.427 ? 0.015  ms/op
AclAuthorizerBenchmark.testAclsIterator           5            50000  avgt   15  21.410 ? 0.225  ms/op
AclAuthorizerBenchmark.testAclsIterator          10             5000  avgt   15   1.230 ? 0.018  ms/op
AclAuthorizerBenchmark.testAclsIterator          10            10000  avgt   15   4.303 ? 0.744  ms/op
AclAuthorizerBenchmark.testAclsIterator          10            50000  avgt   15  36.724 ? 0.409  ms/op
AclAuthorizerBenchmark.testAclsIterator          15             5000  avgt   15   2.433 ? 0.379  ms/op
AclAuthorizerBenchmark.testAclsIterator          15            10000  avgt   15   9.818 ? 0.214  ms/op
AclAuthorizerBenchmark.testAclsIterator          15            50000  avgt   15  52.886 ? 0.525  ms/op
```

Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Author: Lucas Bradstreet <lucas@confluent.io>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>, Lucas Bradstreet <lucas@confluent.io>

Closes #8199 from omkreddy/KAFKA-9626
2020-03-03 01:51:09 +05:30
Manikumar Reddy 922a95a18d KAFKA-9594: Add a separate lock to pause the follower log append while checking if the log dir could be replaced.
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
2020-02-26 10:55:48 +05:30
Lucas Bradstreet 1675115ec1 MINOR: refactor replica last sent HW updates due to performance regression (#7671)
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>
2019-11-12 21:21:18 -08:00
Guozhang Wang 59a75f4422
KAFKA-9048 Pt1: Remove Unnecessary lookup in Fetch Building (#7576)
Get rid of partitionStates that creates a new PartitionState for each state since all the callers do not require it to be a Seq.

Modify ReplicaFetcherThread constructor to fix the broken benchmark path.

This PR:

Benchmark                                  (partitionCount)  Mode  Cnt        Score       Error  Units
ReplicaFetcherThreadBenchmark.testFetcher               100  avgt   15     9280.953 ±    55.967  ns/op
ReplicaFetcherThreadBenchmark.testFetcher               500  avgt   15    61533.546 ±  1213.559  ns/op
ReplicaFetcherThreadBenchmark.testFetcher              1000  avgt   15   151306.146 ±  1820.222  ns/op
ReplicaFetcherThreadBenchmark.testFetcher              5000  avgt   15  1138547.929 ± 45301.938  ns/op

Trunk:

Benchmark                                  (partitionCount)  Mode  Cnt        Score       Error  Units |   |   |   |   |  
-- | -- | -- | -- | -- | --
ReplicaFetcherThreadBenchmark.testFetcher               100  avgt   15     9305.588 ±    51.886  ns/op |   |   |   |   |  
ReplicaFetcherThreadBenchmark.testFetcher               500  avgt   15    65216.933 ±   939.827  ns/op |   |   |   |   |  
ReplicaFetcherThreadBenchmark.testFetcher              1000  avgt   15   151715.514 ±  1361.009  ns/op |   |   |   |   |  
ReplicaFetcherThreadBenchmark.testFetcher              5000  avgt   15  1231958.103 ± 94


Reviewers: Jason Gustafson <jason@confluent.io>, Lucas Bradstreet <lucasbradstreet@gmail.com>
2019-10-28 07:57:50 -07:00
Lucas Bradstreet 8966d066bd KAFKA-9039: Optimize ReplicaFetcher fetch path (#7443)
Improves the performance of the replica fetcher for high partition count fetch requests, where a majority of the partitions did not update between fetch requests. All benchmarks were run on an r5x.large.

Vanilla
Benchmark (partitionCount) Mode Cnt Score Error Units
ReplicaFetcherThreadBenchmark.testFetcher 100 avgt 15 26491.825 ± 438.463 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 500 avgt 15 153941.952 ± 4337.073 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 1000 avgt 15 339868.602 ± 4201.462 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 5000 avgt 15 2588878.448 ± 22172.482 ns/op

From 100 to 5000 partitions the latency increase is 2588878.448 / 26491.825 = 97.

Avoid gettimeofdaycalls in steady state fetch states
8545888

Benchmark (partitionCount) Mode Cnt Score Error Units
ReplicaFetcherThreadBenchmark.testFetcher 100 avgt 15 22685.381 ± 267.727 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 500 avgt 15 113622.521 ± 1854.254 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 1000 avgt 15 273698.740 ± 9269.554 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 5000 avgt 15 2189223.207 ± 1706.945 ns/op

From 100 to 5000 partitions the latency increase is 2189223.207 / 22685.381 = 97X

Avoid copying partition states to maintain fetch offsets
29fdd60

Benchmark (partitionCount) Mode Cnt Score Error Units
ReplicaFetcherThreadBenchmark.testFetcher 100 avgt 15 17039.989 ± 609.355 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 500 avgt 15 99371.086 ± 1833.256 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 1000 avgt 15 216071.333 ± 3714.147 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 5000 avgt 15 2035678.223 ± 5195.232 ns/op

From 100 to 5000 partitions the latency increase is 2035678.223 / 17039.989 = 119X

Keep lag alongside PartitionFetchState to avoid expensive isReplicaInSync check
0e57e3e

Benchmark (partitionCount) Mode Cnt Score Error Units
ReplicaFetcherThreadBenchmark.testFetcher 100 avgt 15 15131.684 ± 382.088 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 500 avgt 15 86813.843 ± 3346.385 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 1000 avgt 15 193050.381 ± 3281.833 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 5000 avgt 15 1801488.513 ± 2756.355 ns/op

From 100 to 5000 partitions the latency increase is 1801488.513 / 15131.684 = 119X

Fetch session optimizations (mostly presizing the next hashmap, and avoiding making a copy of sessionPartitions, as a deep copy is not required for the ReplicaFetcher)
2614b24

Benchmark (partitionCount) Mode Cnt Score Error Units
ReplicaFetcherThreadBenchmark.testFetcher 100 avgt 15 11386.203 ± 416.701 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 500 avgt 15 60820.292 ± 3163.001 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 1000 avgt 15 146242.158 ± 1937.254 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 5000 avgt 15 1366768.926 ± 3305.712 ns/op

From 100 to 5000 partitions the latency increase is 1366768.926 / 11386.203 = 120

Reviewers: Jun Rao <junrao@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
2019-10-16 09:49:53 -07:00
Ismael Juma 66183f730f
KAFKA-8471: Replace control requests/responses with automated protocol (#7353)
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>
2019-09-28 19:39:45 -07:00
Lucas Bradstreet f3ded39a05 KAFKA-8841; Reduce overhead of ReplicaManager.updateFollowerFetchState (#7324)
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>
2019-09-18 09:11:39 -07:00
Lucas Bradstreet fb381cb6c7 MINOR: Fix integer overflow in LRUCacheBenchmark (#7270)
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>
2019-08-29 18:02:27 -07:00
Guozhang Wang 3e9d1c1411
KAFKA-8106: Skipping ByteBuffer allocation of key / value / headers in logValidator (#6785)
* 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
2019-06-21 12:44:45 -07:00
Lifei Chen 3322439d98 MINOR: Document improvement (#6682)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-05-06 16:52:23 +05:30
Ismael Juma 7d9e93ac6d
MINOR: Use https instead of http in links (#6477)
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>
2019-04-22 11:58:25 -07:00
Ismael Juma 12f310d50e
KAFKA-7612: Fix javac warnings and enable warnings as errors (#5900)
- 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>
2018-11-12 22:18:59 -08:00
Armin Braun 346d0ca538 MINOR: Fix needless GC + Result time unit in JMH
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
2017-09-18 10:52:54 +01:00
Bill Bejeck b6adb2dc89 KAFKA-3989; MINOR: follow-up: update script to run from kafka root
…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
2017-08-26 16:15:40 -07:00
Jason Gustafson f49697a279 KAFKA-5456; Ensure producer handles old format large compressed messages
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
2017-06-16 20:36:28 +01:00
Ismael Juma 39eb31feae MINOR: Optimise performance of `Topic.validate()`
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
2017-06-06 03:08:40 +01:00
Ismael Juma c7bc8f7d8c MINOR: Remove redundant volatile write in RecordHeaders
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
2017-06-04 10:48:34 -07:00
Xavier Léauté c060c48285 KAFKA-5150; Reduce LZ4 decompression overhead
- reuse decompression buffers in consumer Fetcher
- switch lz4 input stream to operate directly on ByteBuffers
- avoids performance impact of catching exceptions when reaching the end of legacy record batches
- more tests with both compressible / incompressible data, multiple
  blocks, and various other combinations to increase code coverage
- fixes bug that would cause exception instead of invalid block size
  for invalid incompressible blocks
- fixes bug if incompressible flag is set on end frame block size

Overall this improves LZ4 decompression performance by up to 40x for small batches.
Most improvements are seen for batches of size 1 with messages on the order of ~100B.
We see at least 2x improvements for for batch sizes of < 10 messages, containing messages < 10kB

This patch also yields 2-4x improvements on v1 small single message batches for other compression types.

Full benchmark results can be found here
https://gist.github.com/xvrl/05132e0643513df4adf842288be86efd

Author: Xavier Léauté <xavier@confluent.io>
Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>

Closes #2967 from xvrl/kafka-5150
2017-05-31 02:22:07 +01:00
bbejeck 79f85039d7 KAFKA-3989; Initial support for adding a JMH benchmarking module
Author: bbejeck <bbejeck@gmail.com>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>, Ismael Juma <ismael@juma.me.uk>

Closes #1712 from bbejeck/KAFKA-3989_create_jmh_benchmarking_module
2017-03-06 11:56:14 +00:00