mirror of https://github.com/apache/kafka.git
252 Commits
Author | SHA1 | Message | Date |
---|---|---|---|
|
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> |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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. |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
bb34c5c8cc
|
KAFKA-10350: add forwarding manager implementation with metrics (#9580)
add forwarding manager implementation with metrics Reviewers: David Arthur <mumrah@gmail.com> |
|
|
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> |
|
|
9f26906fcc
|
Revert "KAFKA-9705 part 1: add KIP-590 request header fields (#9144)" (#9523)
This reverts commit
|
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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. |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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 ( |
|
|
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> |
|
|
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.   Reviewers: Ismael Juma <ismael@juma.me.uk> |
|
|
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> |
|
|
46540eb5e0
|
KAFKA-9820: validateMessagesAndAssignOffsetsCompressed allocates unused iterator (#8422)
|
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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 |
|
|
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 |
|
|
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 |
|
|
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
|
|
|
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> |
|
|
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 |
|
|
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> |
|
|
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> |
|
|
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> |
|
|
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 |
|
|
3322439d98 |
MINOR: Document improvement (#6682)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com> |
|
|
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> |
|
|
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> |
|
|
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 |
|
|
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 |
|
|
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 |
|
|
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 |
|
|
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 |
|
|
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 |
|
|
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 |