Commit Graph

205 Commits

Author SHA1 Message Date
Calvin Liu 755adf8a56
KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests (#17698)
Removes the client side AddPartitionsToTxn/AddOffsetsToTxn calls so that the partition is implicitly added as part of KIP-890 part 2. 

This change also requires updating the valid state transitions. The client side can not know for certain if a partition has been added server side when the request times out (partial completion). Thus for TV2, the transition to PrepareAbort is now valid for Empty, CompleteCommit, and CompleteAbort. 

For readability, the V1 and V2 endTransaction methods have been separated. 

Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>, Ritika Reddy <rreddy@confluent.io>
2024-12-06 09:00:04 -08:00
David Jacot 24dd11d693
KAFKA-17593; [8/N] Resolve regular expressions (#17864)
This patch introduces the asynchronous resolution of regular expressions. Let me unpack a few details about the implementations:
1) I have decided to finally update all the regular expressions within a consumer group together. My assumption is that the number of regular expressions in a group will be generally small but the number of topics in a cluster is large. Hence grouping has two benefits. Firstly, it allows to go through the list of topics once for all the regular expressions. Secondly, it reduces the number of potential rebalances because all the regular expressions are updated at the same time.
2) An update is triggered when the group is subscribed to at least one regular expressions.
3) An update is triggered when there is no ongoing update.
4) An update is triggered only of the previous one is older than 10s.
5) An update is triggered when the group has unresolved regular expressions.
6) An update is triggered when the metadata image has new topics.

Reviewers: Jeff Kim <jeff.kim@confluent.io>
2024-11-26 08:56:25 -08:00
TengYao Chi 0e4d8b3e86
KAFKA-17569 Rewrite TestLinearWriteSpeed by Java (#17736)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-26 23:43:01 +08:00
Manikumar Reddy 3268435fd6
KAFKA-18013: Add AutoOffsetResetStrategy internal class (#17858)
- Deprecates OffsetResetStrategy enum
- Adds new internal class AutoOffsetResetStrategy
- Replaces all OffsetResetStrategy enum usages with AutoOffsetResetStrategy
- Deprecate old/Add new constructors to MockConsumer

 Reviewers: Andrew Schofield <aschofield@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2024-11-25 19:11:12 +05:30
Joao Pedro Fonseca Dantas e9ccc2d6f5
KAFKA-16041: Replace Afterburn module with Blackbird (#17884)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-11-21 14:52:45 +01:00
David Jacot a802865aad
KAFKA-17593; [5/N] Include resolved regular expressions into target assignment computation (#17750)
This patch does a few things:
* Refactors the `TargetAssignmentBuilder` to use inheritance to differentiate Consumer and Share groups.
* Introduces `UnionSet` to lazily aggregate the subscriptions for a given member. 
* Wires the resolved regular expressions in the `GroupMetadataManager`. At the moment, they are only used when the target assignment is computed.

Reviewers: Sean Quah <squah@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
2024-11-13 06:59:52 -08:00
TengYao Chi 4e3a3d398d
KAFKA-17570 Rewrite StressTestLog by Java (#17249)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-09 14:24:32 +08:00
Mickael Maison 0049b967e5
KAFKA-17890: Move DelayedOperationPurgatory to server-common (#17636)
Reviewers: Jun Rao <jun@confluent.io>, Apoorv Mittal <amittal@confluent.io>
2024-11-08 09:55:09 +01:00
PoAn Yang 7fb6e9ec1c
KAFKA-17840 Move ReplicationQuotaManager, ClientRequestQuotaManager and QuotaFactory to server module (#17609)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-30 21:18:28 +08:00
Said Boudjelda 57053ef47d
MINOR: Remove never thrown exception in ByteUtilsBenchmark (#17532)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-10-24 11:51:23 +02:00
Ken Huang 2ff13976ab
KAFKA-17568 Rewrite TestPurgatoryPerformance by Java (#17246)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-24 02:44:37 +08:00
Apoorv Mittal 25a3590dc2
KAFKA-17813: Moving broker endpoint class and common server connection id (#17519)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Kuan-Po Tseng <brandboat@gmail.com>, Jun Rao <junrao@gmail.com>
2024-10-22 11:58:28 -07:00
Ken Huang 76a9df47ca
KAFKA-17639 Add Java 23 to CI build matrix (#17409)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-20 23:55:19 +08:00
Abhinav Dixit cb3b03377d
KAFKA-17742: Move DelayedShareFetchPurgatory declaration to ReplicaManager (#17437)
Declare the delayed share fetch purgatory inside ReplicaManager along with the existing purgatories. 

Check the share fetch purgatory when a replica becomes the follower or a replica is deleted from a broker through ReplicaManager.

Perform a checkAndComplete for share fetch when HWM is updated.

Reviewers:  Andrew Schofield <aschofield@confluent.io>,  Apoorv Mittal <apoorvmittal10@gmail.com>, Jun Rao <junrao@gmail.com>
2024-10-17 13:58:10 -07:00
TengYao Chi 582bb48e88
KAFKA-17748 Remove scala-java8-compat (#17497)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-15 13:34:21 +08:00
Linsiyuan9 76a1af984b
KAFKA-17746 Replace JavaConverters with CollectionConverters (#17451)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-14 17:13:20 +08:00
Mickael Maison 07cafdd9df
KAFKA-17729: Remove ZK from AuthorizerBenchmark, CheckpointBench and PartitionCreationBench (#17415)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-09 11:07:15 +08:00
Ken Huang 10a0905628
KAFKA-17564 Move BrokerFeatures to server module (#17228)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-10-07 15:16:48 +08:00
TengYao Chi 0e4eebe9c0
KAFKA-12895 Drop support for Scala 2.12 in Kafka 4.0 (#17313)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-07 01:34:38 +08:00
Colin Patrick McCabe 85bfdf4127
KAFKA-17613: Remove ZK migration code (#17293)
Remove the controller machinery for doing ZK migration in Kafka 4.0.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2024-10-03 12:01:14 -07:00
Sean Quah 99e1d8fbb3
MINOR: Cache topic resolution in TopicIds set (#17285)
Looking up topics in a TopicsImage is relatively slow. Cache the results
in TopicIds to improve assignor performance. In benchmarks, we see a
noticeable improvement in performance in the heterogeneous case.

Before
```
Benchmark                                       (assignmentType)  (assignorType)  (isRackAware)  (memberCount)  (partitionsToMemberRatio)  (subscriptionType)  (topicCount)  Mode  Cnt    Score   Error  Units
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false          10000                         10         HOMOGENEOUS          1000  avgt    5   36.400 ± 3.004  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false          10000                         10       HETEROGENEOUS          1000  avgt    5  158.340 ± 0.825  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL         UNIFORM          false          10000                         10         HOMOGENEOUS          1000  avgt    5    1.329 ± 0.041  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL         UNIFORM          false          10000                         10       HETEROGENEOUS          1000  avgt    5  382.901 ± 6.203  ms/op
```

After
```
Benchmark                                       (assignmentType)  (assignorType)  (isRackAware)  (memberCount)  (partitionsToMemberRatio)  (subscriptionType)  (topicCount)  Mode  Cnt    Score   Error  Units
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false          10000                         10         HOMOGENEOUS          1000  avgt    5   36.465 ± 1.954  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false          10000                         10       HETEROGENEOUS          1000  avgt    5  114.043 ± 1.424  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL         UNIFORM          false          10000                         10         HOMOGENEOUS          1000  avgt    5    1.454 ± 0.019  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL         UNIFORM          false          10000                         10       HETEROGENEOUS          1000  avgt    5  342.840 ± 2.744  ms/op
```

---

Based heavily on https://github.com/apache/kafka/pull/16527.

Reviewers: David Arthur <mumrah@gmail.com>, David Jacot <djacot@confluent.io>
2024-10-03 00:40:25 -07:00
Dimitar Dimitrov bc47ce1a53
MINOR: Fix a race and add JMH bench for HdrHistogram (#17221) 2024-09-27 23:49:10 +09:00
xijiu 18340c9733
KAFKA-17563 Move `RequestConvertToJson` to server module (#17223)
Reviewers: Chia-Ping Tsai <chia7712@apache.org>
2024-09-27 02:19:47 +08:00
Sean Quah 236f3d422f
KAFKA-17496: Add heterogeneous case to TargetAssignmentBuilderBenchmark (#17277)
Bring the homogeneous case from ServerSideAssignorBenchmark to TargetAssignmentBuilderBenchmark.

Reviewers: David Jacot <djacot@confluent.io>
2024-09-25 23:59:38 -07:00
PoAn Yang bb97d63d41
KAFKA-17578: Remove partitionRacks from TopicMetadata (#17233)
The ModernGroup#subscribedTopicMetadata takes too much memory due to partitionRacks. This is not being used at the moment as the consumer protocol does not support rack aware assignments.

A heap dump from a group with 500 members, 2K subscribed topic partitions shows 654,400 bytes used for partitionRacks. The rest of the ConsumerGroup object holds 822,860 bytes.

Reviewers: David Jacot <djacot@confluent.io>
2024-09-25 00:48:48 -07:00
Sean Quah 9352faa8fc
KAFKA-17495: Factor out assignor benchmark code into utils class (#17133)
ServerSideAssignorBenchmark and TargetAssignmentBuilderBenchmark have
the same topic and member subscription setup for the most part. Factor
out the commonality so that it's easier to share new setups between both
benchmarks.

Reviewers: David Jacot <djacot@confluent.io>
2024-09-23 07:55:54 -07:00
Dmitry Werner 5fd7ce2ace
KAFKA-17414 Move RequestLocal to server-common module (#16986)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-04 16:12:20 +08:00
Mickael Maison c30615e6d7
KAFKA-17430: Move RequestChannel.Metrics/RequestMetrics to server module (#17015)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-03 10:11:47 +02:00
Mickael Maison b9fe9f532f
KAFKA-16972: Move BrokerTopicStats to storage module (#17003)
Reviewers: Luke Chen <showuon@gmail.com>
2024-08-27 11:39:37 +02:00
TengYao Chi d67c18b4ae
KAFKA-17331 Set correct version for EarliestLocalSpec and LatestTieredSpec (#16876)
Add the version check to client side when building ListOffsetRequest for the specific timestamp:
1) the version must be >=8 if timestamp=-4L (EARLIEST_LOCAL_TIMESTAMP)
2) the version must be >=9 if timestamp=-5L (LATEST_TIERED_TIMESTAMP)

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-25 17:39:28 +08:00
Mickael Maison e23172a48a
MINOR: Move OffsetCheckpointFile to storage module (#16917)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-20 16:29:24 +02:00
David Schlosnagle 050edfaf00
KAFKA-14336: MetadataResponse#convertToNodeArray uses iteration (#12782)
Avoids stream allocation on hot code path in Admin#listOffsets

This patch avoids allocating the stream reference pipeline & spliterator for this case by explicitly allocating the pre-sized Node[] and using a for loop with int induction over the specified IDs List argument.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Kirk True <kirk@kirktrue.pro>, David Arthur <mumrah@gmail.com>
2024-08-19 19:46:51 -04:00
Josep Prat 4e862c0903
KAFKA-15875: Stops leak Snapshot in public methods (#16807)
* KAFKA-15875: Stops leak Snapshot in public methods

The Snapshot class is package protected but it's returned in
several public methods in SnapshotRegistry.
To prevent this accidental leakage, these methods are made
package protected as well. For getOrCreateSnapshot a new
method called IdempotentCreateSnapshot is created that returns void.
* Make builer package protected, replace <br> with <p>

Reviewers: Greg Harris <greg.harris@aiven.io>
2024-08-08 20:05:47 +02:00
Chirag Wadhwa 1db84c1a11
KAFKA-16745: Implemented handleShareFetchRequest RPC including unit tests (#16456)
Implemented handleShareFetch request RPC in KafkaApis.scala. This method is called whenever the client sends a Share Fetch request to the broker. Although Share Fetch request support acknowledgements, since the logic for acknowledging records is not completely implemented in SharePartitionManager.java class, this method currently includes placeholder code for acknowledging, which will be replaced by the actual functionality in the upcoming PRs.

Reviewers:  Apoorv Mittal <apoorvmittal10@gmail.com>, Abhinav Dixit <adixit@confluent.io>, Jun Rao <junrao@gmail.com>
2024-08-06 07:59:04 -07:00
PoAn Yang 6e324487fa
KAFKA-16480: ListOffsets change should have an associated API/IBP version update (#16781)
1. Use oldestAllowedVersion as 9 if using ListOffsetsRequest#EARLIEST_LOCAL_TIMESTAMP or ListOffsetsRequest#LATEST_TIERED_TIMESTAMP.
   2. Add test cases to ListOffsetsRequestTest#testListOffsetsRequestOldestVersion to make sure requireTieredStorageTimestamp return 9 as minVersion.
   3. Add EarliestLocalSpec and LatestTierSpec to OffsetSpec.
   4. Add more cases to KafkaAdminClient#getOffsetFromSpec.
   5. Add testListOffsetsEarliestLocalSpecMinVersion and testListOffsetsLatestTierSpecSpecMinVersion to KafkaAdminClientTest to make sure request builder has oldestAllowedVersion as 9.

Signed-off-by: PoAn Yang <payang@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>
2024-08-03 14:27:27 +08:00
Colin Patrick McCabe 4d3e366bc2
KAFKA-16772: Introduce kraft.version to support KIP-853 (#16230)
Introduce the KRaftVersion enum to describe the current value of kraft.version. Change a bunch of places in the code that were using raw shorts over to using this new enum.

In BrokerServer.scala, fix a bug that could cause null pointer exceptions during shutdown if we tried to shut down before fully coming up.

Do not send finalized features that are finalized as level 0, since it is a no-op.

Reviewers: dengziming <dengziming1993@gmail.com>, José Armando García Sancio <jsancio@apache.org>
2024-07-16 09:31:10 -07:00
Ritika Reddy 42f267a853
KAFKA-16944; Rewrite Range Assignor (#16504)
The server side range assignor was made to be sticky i.e. partitions from the existing assignment are retained as much as possible. During a rebalance, the expected behavior is to achieve co-partitioning for members that are subscribed to the same set of topics with equal number of partitions.

However, there are cases where this cannot be achieved efficiently with the current algorithm. There is no easy way to implement stickiness and co-partitioning and hence we have resorted to recomputing the target assignment every time.

In case of static membership, instanceIds are leveraged to ensure some form of stickiness.

```
Benchmark                                       (assignmentType)  (assignorType)  (isRackAware)  (memberCount)  (partitionsToMemberRatio)  (subscriptionType)  (topicCount)  Mode  Cnt    Score    Error  Units
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false            100                         10         HOMOGENEOUS           100  avgt    5    0.052 ±  0.001  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false            100                         10         HOMOGENEOUS          1000  avgt    5    0.454 ±  0.003  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false           1000                         10         HOMOGENEOUS           100  avgt    5    0.476 ±  0.046  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false           1000                         10         HOMOGENEOUS          1000  avgt    5    3.102 ±  0.055  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false          10000                         10         HOMOGENEOUS           100  avgt    5    5.640 ±  0.223  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false          10000                         10         HOMOGENEOUS          1000  avgt    5   37.947 ±  1.000  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false            100                         10       HETEROGENEOUS           100  avgt    5    0.172 ±  0.001  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false            100                         10       HETEROGENEOUS          1000  avgt    5    1.882 ±  0.006  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false           1000                         10       HETEROGENEOUS           100  avgt    5    1.730 ±  0.036  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false           1000                         10       HETEROGENEOUS          1000  avgt    5   17.654 ±  1.160  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false          10000                         10       HETEROGENEOUS           100  avgt    5   18.595 ±  0.316  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL           RANGE          false          10000                         10       HETEROGENEOUS          1000  avgt    5  172.398 ±  2.251  ms/op
JMH benchmarks done

Benchmark                                     (memberCount)  (partitionsToMemberRatio)  (topicCount)  Mode  Cnt   Score   Error  Units
TargetAssignmentBuilderBenchmark.build                  100                         10           100  avgt    5   0.071 ± 0.004  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         10          1000  avgt    5   0.428 ± 0.026  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         10           100  avgt    5   0.659 ± 0.028  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         10          1000  avgt    5   3.346 ± 0.102  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10           100  avgt    5   8.947 ± 0.386  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10          1000  avgt    5  40.240 ± 3.113  ms/op
JMH benchmarks done
```

Reviewers: David Jacot <djacot@confluent.io>
2024-07-04 10:33:09 -07:00
Apoorv Mittal f2dbc55d24
KAFKA-17047: Refactored group coordinator classes to modern package (KIP-932) (#16474)
Following the discussion and suggestion by @dajac, https://github.com/apache/kafka/pull/16054#discussion_r1613638293, the PR refactors the common classes to build TargetAssignment in `modern` package. `consumer` package has been moved inside `modern` package with classes exclusive to `consumer group`.

This PR completes the refactoring and base to introduce `share` package inside `modern`. The subsequent PRs will define the implementation specific to Share Groups while re-using the common functionality from `modern` package classes. 

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
2024-07-03 00:16:40 -07:00
Apoorv Mittal 60114a46a7
KAFKA-16822: Abstract consumer group to share functionality with share group (KIP-932) (#16054)
Abstracted code for 2 classes `ConsumerGroup` and `ConsumerGroupMember` to `ModernGroup` and `ModernGroupMember` respectively. The new abstract classes are created to share common functionality with `ShareGroup` and `ShareGroupMember` which are being introduced with KIP-932.

The patch is majorly code refactoring from existing classes to abstract classes. Also created a new package called `modern` where `MemberState` class is moved, in upcoming patches, I will move common classes for `Share` and `Consumer` Group in `modern` package itself. 

Reviewers: Lianet Magrans <lianetmr@gmail.com>, Andrew Schofield <aschofield@confluent.io>, David Jacot <djacot@confluent.io>
2024-06-27 05:42:58 -07:00
Kuan-Po (Cooper) Tseng 888a177603
KAFKA-12708 Rewrite org.apache.kafka.test.Microbenchmarks by JMH (#16231)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-14 16:47:34 +08:00
gongxuanzhang 596b945072
KAFKA-16643 Add ModifierOrder checkstyle rule (#15890)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-13 15:39:32 +08:00
gongxuanzhang 46eb0814f6
KAFKA-10787 Apply spotless to log4j-appender, trogdor, jmh-benchmarks, examples, shell and generator (#16296)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-12 22:23:39 +08:00
David Jacot 049cfeac02
MINOR: Rename uniform assignor's internal builders (#16233)
This patch renames the uniform assignor's builders to match the `SubscriptionType` which is used to determine which one is called. It removes the abstract class `AbstractUniformAssignmentBuilder` which is not necessary anymore. It also applies minor refactoring.

Reviewers: Ritika Reddy <rreddy@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-10 05:26:56 -07:00
David Jacot 7d832cf74f
KAFKA-14701; Move `PartitionAssignor` to new `group-coordinator-api` module (#16198)
This patch moves the `PartitionAssignor` interface and all the related classes to a newly created `group-coordinator/api` module, following the pattern used by the storage and tools modules.

Reviewers: Ritika Reddy <rreddy@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-06 12:19:20 -07:00
Ritika Reddy 078dd9a311
KAFKA-16821; Member Subscription Spec Interface (#16068)
This patch reworks the `PartitionAssignor` interface to use interfaces instead of POJOs. It mainly introduces the `MemberSubscriptionSpec` interface that represents a member subscription and changes the `GroupSpec` interfaces to expose the subscriptions and the assignments via different methods.

The patch does not change the performance.

before:
```
Benchmark                                     (memberCount)  (partitionsToMemberRatio)  (topicCount)  Mode  Cnt  Score   Error  Units
TargetAssignmentBuilderBenchmark.build                10000                         10           100  avgt    5  3.462 ± 0.687  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10          1000  avgt    5  3.626 ± 0.412  ms/op
JMH benchmarks done
```

after:
```
Benchmark                                     (memberCount)  (partitionsToMemberRatio)  (topicCount)  Mode  Cnt  Score   Error  Units
TargetAssignmentBuilderBenchmark.build                10000                         10           100  avgt    5  3.677 ± 0.683  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10          1000  avgt    5  3.991 ± 0.065  ms/op
JMH benchmarks done
```

Reviewers: David Jacot <djacot@confluent.io>
2024-06-04 06:44:37 -07:00
David Jacot fb566e48bf
KAFKA-16864; Optimize uniform (homogenous) assignor (#16088)
This patch optimizes uniform (homogenous) assignor by avoiding creating a copy of all the assignments. Instead, the assignor creates a copy only if the assignment is updated. It is a sort of copy-on-write. This change reduces the overhead of the TargetAssignmentBuilder when ran with the uniform (homogenous) assignor.

Trunk:

```
Benchmark                                     (memberCount)  (partitionsToMemberRatio)  (topicCount)  Mode  Cnt   Score   Error  Units
TargetAssignmentBuilderBenchmark.build                10000                         10           100  avgt    5  24.535 ± 1.583  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10          1000  avgt    5  24.094 ± 0.223  ms/op
JMH benchmarks done
```

```
Benchmark                                       (assignmentType)  (assignorType)  (isRackAware)  (memberCount)  (partitionsToMemberRatio)  (subscriptionType)  (topicCount)  Mode  Cnt   Score   Error  Units
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL         UNIFORM          false          10000                         10         HOMOGENEOUS           100  avgt    5  14.697 ± 0.133  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL         UNIFORM          false          10000                         10         HOMOGENEOUS          1000  avgt    5  15.073 ± 0.135  ms/op
JMH benchmarks done
```

Patch:

```
Benchmark                                     (memberCount)  (partitionsToMemberRatio)  (topicCount)  Mode  Cnt  Score   Error  Units
TargetAssignmentBuilderBenchmark.build                10000                         10           100  avgt    5  3.376 ± 0.577  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10          1000  avgt    5  3.731 ± 0.359  ms/op
JMH benchmarks done
```

```
Benchmark                                       (assignmentType)  (assignorType)  (isRackAware)  (memberCount)  (partitionsToMemberRatio)  (subscriptionType)  (topicCount)  Mode  Cnt  Score   Error  Units
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL         UNIFORM          false          10000                         10         HOMOGENEOUS           100  avgt    5  1.975 ± 0.086  ms/op
ServerSideAssignorBenchmark.doAssignment             INCREMENTAL         UNIFORM          false          10000                         10         HOMOGENEOUS          1000  avgt    5  2.026 ± 0.190  ms/op
JMH benchmarks done
```

Reviewers: Ritika Reddy <rreddy@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
2024-05-31 13:17:59 -07:00
Mickael Maison 8068a086a3
MINOR: Remove KafkaConfig dependency in KafkaRequestHandler (#16108)
Reviewers: Luke Chen <showuon@gmail.com>, Apoorv Mittal <amittal@confluent.io>
2024-05-30 11:51:24 +02:00
Calvin Liu c8af740bd4
Improve producer ID expiration performance (#16075)
Skip using stream when expiring the producer ID. This can improve the performance significantly when the count is high.
Before

Benchmark                                        (numProducerIds)  Mode  Cnt      Score       Error  Units
ProducerStateManagerBench.testDeleteExpiringIds             10000  avgt    3    101.253 ±    28.031  us/op
ProducerStateManagerBench.testDeleteExpiringIds            100000  avgt    3   2297.219 ±  1690.486  us/op
ProducerStateManagerBench.testDeleteExpiringIds           1000000  avgt    3  30688.865 ± 16348.768  us/op
After

Benchmark                                        (numProducerIds)  Mode  Cnt     Score     Error  Units
ProducerStateManagerBench.testDeleteExpiringIds             10000  avgt    3    39.122 ±   1.151  us/op
ProducerStateManagerBench.testDeleteExpiringIds            100000  avgt    3   464.363 ±  98.857  us/op
ProducerStateManagerBench.testDeleteExpiringIds           1000000  avgt    3  5731.169 ± 674.380  us/op
Also, made a change to the JMH testing which excludes the producer ID populating from the testing.

Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>
2024-05-29 16:49:55 -07:00
Justine Olshan 5e3df22095
KAFKA-16308 [1/N]: Create FeatureVersion interface and add `--feature` flag and handling to StorageTool (#15685)
As part of KIP-1022, I have created an interface for all the new features to be used when parsing the command line arguments, doing validations, getting default versions, etc.

I've also added the --feature flag to the storage tool to show how it will be used.

Created a TestFeatureVersion to show an implementation of the interface (besides MetadataVersion which is unique) and added tests using this new test feature.

I will add the unstable config and tests in a followup.

Reviewers: David Mao <dmao@confluent.io>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jun Rao <junrao@apache.org>
2024-05-29 16:36:06 -07:00
Omnia Ibrahim 64f699aeea
KAFKA-15853: Move general configs out of KafkaConfig (#16040)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-28 16:22:54 +02:00