Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-16626: Lazily convert subscribed topic names to topic ids #15970

Merged
merged 5 commits into from
May 24, 2024

Conversation

jeffkbkim
Copy link
Contributor

This patch aims to remove the data structure that stores the conversion from topic names to topic ids which was taking time similar to the actual assignment computation.

Instead, we reuse the already existing ConsumerGroupMember.subscribedTopicNames() and do the conversion to topic ids when the iterator is requested.

I have provided the results from trunk vs. this PR in the comments

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@jeffkbkim
Copy link
Contributor Author

jeffkbkim commented May 15, 2024

TargetAssignmentBuilderBenchmark results (Baseline vs KAFKA-16626). The runs with high consumer group member count is noticeably better now.

Baseline

Benchmark                                     (memberCount)  (partitionsToMemberRatio)  (topicCount)  Mode  Cnt    Score    Error  Units
TargetAssignmentBuilderBenchmark.build                  100                          5            10  avgt    5    0.181 ±  0.025  ms/op
TargetAssignmentBuilderBenchmark.build                  100                          5           100  avgt    5    0.483 ±  0.042  ms/op
TargetAssignmentBuilderBenchmark.build                  100                          5          1000  avgt    5    3.693 ±  0.074  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         10            10  avgt    5    0.287 ±  0.010  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         10           100  avgt    5    0.634 ±  0.034  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         10          1000  avgt    5    4.573 ±  0.127  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         50            10  avgt    5    1.016 ±  0.063  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         50           100  avgt    5    1.960 ±  0.106  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         50          1000  avgt    5    6.669 ±  0.257  ms/op
TargetAssignmentBuilderBenchmark.build                  500                          5            10  avgt    5    0.849 ±  0.083  ms/op
TargetAssignmentBuilderBenchmark.build                  500                          5           100  avgt    5    2.209 ±  0.139  ms/op
TargetAssignmentBuilderBenchmark.build                  500                          5          1000  avgt    5   22.515 ±  1.182  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         10            10  avgt    5    1.519 ±  0.181  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         10           100  avgt    5    3.038 ±  0.291  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         10          1000  avgt    5   23.761 ±  0.734  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         50            10  avgt    5    6.651 ±  0.785  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         50           100  avgt    5   16.168 ±  0.939  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         50          1000  avgt    5   39.315 ±  1.811  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                          5            10  avgt    5    1.782 ±  0.193  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                          5           100  avgt    5    4.886 ±  0.669  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                          5          1000  avgt    5   37.605 ±  0.674  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         10            10  avgt    5    3.982 ±  0.534  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         10           100  avgt    5    7.444 ±  0.411  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         10          1000  avgt    5   46.231 ±  1.616  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         50            10  avgt    5   15.327 ±  1.394  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         50           100  avgt    5   29.014 ±  0.890  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         50          1000  avgt    5   76.194 ±  0.952  ms/op
TargetAssignmentBuilderBenchmark.build                 5000                          5            10  avgt    5   18.086 ±  2.724  ms/op
TargetAssignmentBuilderBenchmark.build                 5000                          5           100  avgt    5   31.667 ±  1.372  ms/op
TargetAssignmentBuilderBenchmark.build                 5000                          5          1000  avgt    5  208.516 ±  7.774  ms/op
TargetAssignmentBuilderBenchmark.build                 5000                         10            10  avgt    5   25.929 ±  1.926  ms/op
TargetAssignmentBuilderBenchmark.build                 5000                         10           100  avgt    5   43.845 ±  1.089  ms/op
TargetAssignmentBuilderBenchmark.build                 5000                         10          1000  avgt    5  248.228 ± 16.409  ms/op
TargetAssignmentBuilderBenchmark.build                 5000                         50            10  avgt    5   87.236 ±  7.133  ms/op
TargetAssignmentBuilderBenchmark.build                 5000                         50           100  avgt    5  150.507 ± 14.060  ms/op
TargetAssignmentBuilderBenchmark.build                 5000                         50          1000  avgt    5  325.549 ±  5.762  ms/op
TargetAssignmentBuilderBenchmark.build                10000                          5            10  avgt    5   32.577 ±  2.574  ms/op
TargetAssignmentBuilderBenchmark.build                10000                          5           100  avgt    5   65.163 ±  7.653  ms/op
TargetAssignmentBuilderBenchmark.build                10000                          5          1000  avgt    5  401.885 ± 23.356  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10            10  avgt    5   56.647 ±  4.140  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10           100  avgt    5   88.750 ± 13.175  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10          1000  avgt    5  484.745 ± 28.352  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         50            10  avgt    5  184.636 ± 13.637  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         50           100  avgt    5  308.914 ± 31.608  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         50          1000  avgt    5  738.403 ± 39.927  ms/op

KAFKA-16626

Benchmark                                     (memberCount)  (partitionsToMemberRatio)  (topicCount)  Mode  Cnt    Score    Error  Units
TargetAssignmentBuilderBenchmark.build                  100                          5            10  avgt    5    0.175 ±  0.015  ms/op
TargetAssignmentBuilderBenchmark.build                  100                          5           100  avgt    5    0.228 ±  0.082  ms/op
TargetAssignmentBuilderBenchmark.build                  100                          5          1000  avgt    5    0.271 ±  0.037  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         10            10  avgt    5    0.293 ±  0.047  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         10           100  avgt    5    0.406 ±  0.068  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         10          1000  avgt    5    0.969 ±  1.345  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         50            10  avgt    5    1.312 ±  0.450  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         50           100  avgt    5    2.227 ±  0.315  ms/op
TargetAssignmentBuilderBenchmark.build                  100                         50          1000  avgt    5    2.900 ±  0.622  ms/op
TargetAssignmentBuilderBenchmark.build                  500                          5            10  avgt    5    1.034 ±  0.306  ms/op
TargetAssignmentBuilderBenchmark.build                  500                          5           100  avgt    5    1.155 ±  0.279  ms/op
TargetAssignmentBuilderBenchmark.build                  500                          5          1000  avgt    5    1.120 ±  0.285  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         10            10  avgt    5    2.237 ±  0.226  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         10           100  avgt    5    2.357 ±  0.068  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         10          1000  avgt    5    2.830 ±  0.401  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         50            10  avgt    5    8.198 ±  0.608  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         50           100  avgt    5   14.060 ±  0.172  ms/op
TargetAssignmentBuilderBenchmark.build                  500                         50          1000  avgt    5   13.978 ±  0.553  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                          5            10  avgt    5    2.416 ±  0.025  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                          5           100  avgt    5    2.647 ±  0.632  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                          5          1000  avgt    5    3.060 ±  0.235  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         10            10  avgt    5    4.811 ±  0.167  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         10           100  avgt    5    4.922 ±  0.439  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         10          1000  avgt    5    6.042 ±  0.422  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         50            10  avgt    5   18.450 ±  0.722  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         50           100  avgt    5   29.552 ±  0.649  ms/op
TargetAssignmentBuilderBenchmark.build                 1000                         50          1000  avgt    5   32.547 ±  1.433  ms/op
TargetAssignmentBuilderBenchmark.build                10000                          5            10  avgt    5   30.797 ±  3.960  ms/op
TargetAssignmentBuilderBenchmark.build                10000                          5           100  avgt    5   31.279 ±  2.803  ms/op
TargetAssignmentBuilderBenchmark.build                10000                          5          1000  avgt    5   31.539 ±  2.764  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10            10  avgt    5   62.082 ±  4.681  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10           100  avgt    5   54.614 ±  1.589  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         10          1000  avgt    5   67.161 ±  2.333  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         50            10  avgt    5  185.549 ±  9.876  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         50           100  avgt    5  287.233 ± 22.054  ms/op
TargetAssignmentBuilderBenchmark.build                10000                         50          1000  avgt    5  283.280 ± 17.492  ms/op

@dajac dajac added the KIP-848 label May 16, 2024
@jeffkbkim
Copy link
Contributor Author

RangeAssignor baseline

Benchmark                                     (memberCount)  (partitionsToMemberRatio)  (topicCount)  Mode  Cnt    Score    Error  Units
TargetAssignmentBuilderBenchmark.build                10000                         10           100  avgt    5  235.398 ± 15.092  ms/op

RangeAssignor PR

Benchmark                                     (memberCount)  (partitionsToMemberRatio)  (topicCount)  Mode  Cnt    Score    Error  Units
TargetAssignmentBuilderBenchmark.build                10000                         10           100  avgt    5  161.683 ± 22.739  ms/op

Copy link

@dungeon-master-nstream dungeon-master-nstream left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@jeffkbkim jeffkbkim force-pushed the KAFKA-16626-subscribedTopicNames branch from e5c7f7f to a7b0551 Compare May 20, 2024 21:21
Copy link
Contributor

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jeffkbkim Thanks for the patch. I left some comments.

@jeffkbkim
Copy link
Contributor Author

@dajac thanks for the review and the suggestions, i was a bit annoyed with the code to create those topics images. I have addressed the comments

Copy link
Contributor

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jeffkbkim Thanks for the update. I left a few more comments.

@@ -103,13 +114,13 @@ public void testTwoMembersSubscribedToNonexistentTopics() {
members.put(memberA, new AssignmentMemberSpec(
Optional.empty(),
Optional.empty(),
Collections.singletonList(topic3Uuid),
new TopicIds(Collections.singleton(topic3Name), topicsImage),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need this change? It looks like the previous version would just work, no?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I removed all TopicIds initializations in the assignor tests.

@jeffkbkim
Copy link
Contributor Author

@dajac thanks for the review. I have addressed your comments

Copy link
Contributor

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jeffkbkim Thanks for the update. I left a few minor comments.

@jeffkbkim
Copy link
Contributor Author

@dajac thanks for the review. I have addressed your comments

Copy link
Contributor

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jeffkbkim Thanks for the update. I left a few nits regarding the code style. I also noticed that the build failed due to checkstyle issues. It may be due to the one that I reported.

Copy link
Contributor

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thanks.

@jeffkbkim jeffkbkim force-pushed the KAFKA-16626-subscribedTopicNames branch from a3b2566 to 3f0232c Compare May 22, 2024 16:04
Copy link
Contributor

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thanks.

@dajac
Copy link
Contributor

dajac commented May 24, 2024

The failed tests are related to #15972. Merging to trunk.

@dajac dajac merged commit 520aa86 into apache:trunk May 24, 2024
1 check failed
apourchet added a commit to apourchet/kafka that referenced this pull request May 29, 2024
commit cc269b0
Author: Antoine Pourchet <antoine@responsive.dev>
Date:   Wed May 29 14:15:02 2024 -0600

    KAFKA-15045: (KIP-924 pt. 14) Callback to TaskAssignor::onAssignmentComputed (apache#16123)

    This PR adds the logic and wiring necessary to make the callback to
    TaskAssignor::onAssignmentComputed with the necessary parameters.

    We also fixed some log statements in the actual assignment error
    computation, as well as modified the ApplicationState::allTasks method
    to return a Map instead of a Set of TaskInfos.

    Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>

commit 862ea12
Author: Eugene Mitskevich <emitskevich@bloomberg.net>
Date:   Wed May 29 16:14:37 2024 -0400

    MINOR: Fix rate metric spikes (apache#15889)

    Rate reports value in the form of sumOrCount/monitoredWindowSize. It has a bug in monitoredWindowSize calculation, which leads to spikes in result values.

    Reviewers: Jun Rao <junrao@gmail.com>

commit 0f0c9ec
Author: gongxuanzhang <gongxuanzhangmelt@gmail.com>
Date:   Thu May 30 01:08:17 2024 +0800

    KAFKA-16771 First log directory printed twice when formatting storage (apache#16010)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit 2d9994e
Author: Andrew Schofield <aschofield@confluent.io>
Date:   Wed May 29 16:31:52 2024 +0100

    KAFKA-16722: Introduce ConsumerGroupPartitionAssignor interface (apache#15998)

    KIP-932 introduces share groups to go alongside consumer groups. Both kinds of group use server-side assignors but it is unlikely that a single assignor class would be suitable for both. As a result, the KIP introduces specific interfaces for consumer group and share group partition assignors.

    This PR introduces only the consumer group interface, `o.a.k.coordinator.group.assignor.ConsumerGroupPartitionAssignor`. The share group interface will come in a later release. The existing implementations of the general `PartitionAssignor` interface have been changed to implement `ConsumerGroupPartitionAssignor` instead and all other code changes are just propagating the change throughout the codebase.

    Note that the code in the group coordinator that actually calculates assignments uses the general `PartitionAssignor` interface so that it can be used with both kinds of group, even though the assignors themselves are specific.

    Reviewers: Apoorv Mittal <amittal@confluent.io>, David Jacot <djacot@confluent.io>

commit 0b75cf7
Author: gongxuanzhang <gongxuanzhangmelt@gmail.com>
Date:   Wed May 29 22:38:00 2024 +0800

    KAFKA-16705 the flag "started" of RaftClusterInstance is false even though the cluster is started (apache#15946)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit 8d11d95
Author: Loïc GREFFIER <loic.greffier@hotmail.fr>
Date:   Wed May 29 14:09:22 2024 +0200

    KAFKA-16448: Add ProcessingExceptionHandler interface and implementations (apache#16090)

    This PR is part of KAFKA-16448 which aims to bring a ProcessingExceptionHandler to Kafka Streams in order to deal with exceptions that occur during processing.

    This PR brings ProcessingExceptionHandler interface and default implementations.

    Co-authored-by: Dabz <d.gasparina@gmail.com>
    Co-authored-by: sebastienviale <sebastien.viale@michelin.com>

    Reviewer: Bruno Cadonna <cadonna@apache.org>

commit b73f479
Author: Ramin Gharib <ramingharib@gmail.com>
Date:   Wed May 29 13:12:54 2024 +0200

    KAFKA-16362: Fix type-unsafety in KStreamKStreamJoin caused by isLeftSide (apache#15601)

    The introduced changes provide a cleaner definition of the join side in KStreamKStreamJoin. Before, this was done by using a Boolean flag, which led to returning a raw LeftOrRightValue without generic arguments because the generic type arguments depended on the boolean input.

    Reviewers: Greg Harris <greg.harris@aiven.io>, Bruno Cadonna <cadonna@apache.org>

commit 897cab2
Author: Luke Chen <showuon@gmail.com>
Date:   Wed May 29 15:30:18 2024 +0800

    KAFKA-16399: Add JBOD support in tiered storage (apache#15690)

    After JBOD is supported in KRaft, we should also enable JBOD support in tiered storage. Unit tests and Integration tests are also added.

    Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Igor Soarez <soarez@apple.com>, Mickael Maison <mickael.maison@gmail.com>

commit eefd114
Author: Dongnuo Lyu <139248811+dongnuo123@users.noreply.github.com>
Date:   Wed May 29 02:21:30 2024 -0400

    KAFKA-16832; LeaveGroup API for upgrading ConsumerGroup (apache#16057)

    This patch implements the LeaveGroup API to the consumer groups that are in the mixed mode.

    Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Jacot <djacot@confluent.io>

commit 9562143
Author: A. Sophie Blee-Goldman <ableegoldman@gmail.com>
Date:   Tue May 28 21:35:02 2024 -0700

    HOTFIX: remove unnecessary list creation (apache#16117)

    Removing a redundant list declaration in the new StickyTaskAssignor implementation

    Reviewers: Antoine Pourchet <antoine@responsive.dev>

commit d64e3fb
Author: Antoine Pourchet <antoine@responsive.dev>
Date:   Tue May 28 20:43:30 2024 -0600

    KAFKA-15045: (KIP-924 pt. 13) AssignmentError calculation added (apache#16114)

    This PR adds the post-processing of the TaskAssignment to figure out if the new assignment is valid, and return an AssignmentError otherwise.

    Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>

commit 8d243df
Author: Antoine Pourchet <antoine@responsive.dev>
Date:   Tue May 28 19:01:18 2024 -0600

    KAFKA-15045: (KIP-924 pt. 12) Wiring in new assignment configs and logic (apache#16074)

    This PR creates the new public config of KIP-924 in StreamsConfig and uses it to instantiate user-created TaskAssignors. If such a TaskAssignor is found and successfully created we then use that assignor to perform the task assignment, otherwise we revert back to the pre KIP-924 world with the internal task assignors.

    Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Almog Gavra <almog@responsive.dev>

commit 56ee139
Author: Antoine Pourchet <antoine@responsive.dev>
Date:   Tue May 28 18:05:51 2024 -0600

    KAFKA-15045: (KIP-924 pt. 11) Implemented StickyTaskAssignor (apache#16052)

    This PR implements the StickyTaskAssignor with the new KIP 924 API.

    Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>

commit 59ba555
Author: Nick Telford <nick.telford@gmail.com>
Date:   Wed May 29 00:23:23 2024 +0100

    KAFKA-15541: Add oldest-iterator-open-since-ms metric (apache#16041)

    Part of [KIP-989](https://cwiki.apache.org/confluence/x/9KCzDw).

    This new `StateStore` metric tracks the timestamp that the oldest
    surviving Iterator was created.

    This timestamp should continue to climb, and closely track the current
    time, as old iterators are closed and new ones created. If the timestamp
    remains very low (i.e. old), that suggests an Iterator has leaked, which
    should enable users to isolate the affected store.

    It will report no data when there are no currently open Iterators.

    Reviewers: Matthias J. Sax <matthias@confluent.io>

commit 4eb60b5
Author: Frederik Rouleau <frouleau@confluent.io>
Date:   Tue May 28 23:56:47 2024 +0200

    KAFKA-16507 Add KeyDeserializationException and ValueDeserializationException with record content (apache#15691)

    Implements KIP-1036.

    Add raw ConsumerRecord data to RecordDeserialisationException to make DLQ implementation easier.

    Reviewers: Kirk True <ktrue@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Matthias J. Sax <matthias@confluent.io>

commit 4d04eb8
Author: PoAn Yang <payang@apache.org>
Date:   Wed May 29 03:13:33 2024 +0800

    KAFKA-16796 Introduce new org.apache.kafka.tools.api.Decoder to replace kafka.serializer.Decoder (apache#16064)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit a649bc4
Author: Luke Chen <showuon@gmail.com>
Date:   Wed May 29 00:05:49 2024 +0800

    KAFKA-16711: Make sure to update highestOffsetInRemoteStorage after log dir change (apache#15947)

    Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>

commit 64f699a
Author: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
Date:   Tue May 28 15:22:54 2024 +0100

    KAFKA-15853: Move general configs out of KafkaConfig (apache#16040)

    Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>

commit 699438b
Author: Sanskar Jhajharia <122860866+sjhajharia@users.noreply.github.com>
Date:   Tue May 28 16:34:44 2024 +0530

    MINOR: Fix the config name in ProducerFailureHandlingTest (apache#16099)

    When moving from KafkaConfig.ReplicaFetchMaxBytesProp we used ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG instead of ReplicationConfigs.REPLICA_FETCH_MAX_BYTES_CONFIG. This PR patches the same.

    Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>

commit a57c05b
Author: Ken Huang <100591800+m1a2st@users.noreply.github.com>
Date:   Tue May 28 17:42:33 2024 +0900

    KAFKA-16805 Stop using a ClosureBackedAction to configure Spotbugs reports (apache#16081)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit 91284d8
Author: Luke Chen <showuon@gmail.com>
Date:   Tue May 28 12:23:34 2024 +0800

    KAFKA-16709: abortAndPauseCleaning only when future log is not existed (apache#15951)

    When doing alter replica logDirs, we'll create a future log and pause log cleaning for the partition( here). And this log cleaning pausing will resume after alter replica logDirs completes (here). And when in the resuming log cleaning, we'll decrement 1 for the LogCleaningPaused count. Once the count reached 0, the cleaning pause is really resuming. (here). For more explanation about the logCleaningPaused state can check here.

    But, there's still one factor that could increase the LogCleaningPaused count: leadership change (here). When there's a leadership change, we'll check if there's a future log in this partition, if so, we'll create future log and pauseCleaning (LogCleaningPaused count + 1). So, if during the alter replica logDirs:

    1. alter replica logDirs for tp0 triggered (LogCleaningPaused count = 1)
    2. tp0 leadership changed (LogCleaningPaused count = 2)
    3. alter replica logDirs completes, resuming logCleaning (LogCleaningPaused count = 1)
    4. LogCleaning keeps paused because the count is always >  0

    This PR fixes this issue by only abortAndPauseCleaning when future log is not existed. We did the same check in alterReplicaLogDirs. So this change can make sure there's only 1 abortAndPauseCleaning for either abortAndPauseCleaning or maybeAddLogDirFetchers. Tests also added.

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Igor Soarez <soarez@apple.com>

commit adab48d
Author: Greg Harris <greg.harris@aiven.io>
Date:   Mon May 27 18:33:01 2024 -0700

    MINOR: Disable JDK 11 and 17 tests on PRs (apache#16051)

    Signed-off-by: Greg Harris <greg.harris@aiven.io>
    Reviewers: Justine Olshan <jolshan@confluent.io>, David Arthur <mumrah@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>

commit bac8df5
Author: Colin P. McCabe <cmccabe@apache.org>
Date:   Mon May 27 08:53:53 2024 -0700

    MINOR: fix typo in KAFKA-16515

commit da3304e
Author: David Jacot <djacot@confluent.io>
Date:   Mon May 27 17:10:37 2024 +0200

    KAFKA-16371; fix lingering pending commit when handling OFFSET_METADATA_TOO_LARGE (apache#16072)

    This patch was initially created in apache#15536.

    When there is a commit for multiple topic partitions and some, but not all, exceed the offset metadata limit, the pending commit is not properly cleaned up leading to UNSTABLE_OFFSET_COMMIT errors when trying to fetch the offsets with read_committed. This change makes it so the invalid commits are not added to the pendingOffsetCommits set.

    Co-authored-by: Kyle Phelps <kyle.phelps@datadoghq.com>

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Justine Olshan <jolshan@confluent.io>

commit 524ad1e
Author: Kamal Chandraprakash <kchandraprakash@uber.com>
Date:   Mon May 27 15:14:23 2024 +0530

    KAFKA-16452: Don't throw OOORE when converting the offset to metadata (apache#15825)

    Don't throw OFFSET_OUT_OF_RANGE error when converting the offset to metadata, and next time the leader should increment the high watermark by itself after receiving fetch requests from followers. This can happen when checkpoint files are missing and being elected as a leader.

    Reviewers: Luke Chen <showuon@gmail.com>, Jun Rao <junrao@apache.org>

commit d9ee9c9
Author: Nick Telford <nick.telford@gmail.com>
Date:   Sat May 25 20:22:56 2024 +0100

    KAFKA-15541: Use LongAdder instead of AtomicInteger (apache#16076)

    `LongAdder` performs better than `AtomicInteger` when under contention
    from many threads. Since it's possible that many Interactive Query
    threads could create a large number of `KeyValueIterator`s, we don't
    want contention on a metric to be a performance bottleneck.

    The trade-off is memory, as `LongAdder` uses more memory to space out
    independent counters across different cache lines. In practice, I don't
    expect this to cause too many problems, as we're only constructing 1
    per-store.

    Reviewers: Matthias J. Sax <matthias@confluent.io>

commit a8d166c
Author: Ritika Reddy <98577846+rreddy-22@users.noreply.github.com>
Date:   Sat May 25 09:06:15 2024 -0700

    KAFKA-16625; Reverse lookup map from topic partitions to members (apache#15974)

    This patch speeds up the computation of the unassigned partitions by exposing the inverted target assignment. It allows the assignor to check whether a partition is assigned or not.

    Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Jacot <djacot@confluent.io>

commit d585a49
Author: Jeff Kim <kimkb2011@gmail.com>
Date:   Fri May 24 16:33:57 2024 -0400

    KAFKA-16831: CoordinatorRuntime should initialize MemoryRecordsBuilder with max batch size write limit (apache#16059)

    CoordinatorRuntime should initialize MemoryRecordsBuilder with max batch size write limit. Otherwise, we default the write limit to the min buffer size of 16384 for the write limit. This causes the coordinator to threw RecordTooLargeException even when it's under the 1MB max batch size limit.

    Reviewers: David Jacot <djacot@confluent.io>

commit 8eea6b8
Author: Edoardo Comar <ecomar@uk.ibm.com>
Date:   Fri May 24 20:33:00 2024 +0100

    MINOR: mention KAFKA-15905 in docs "Notable changes in 3.7.1" (apache#16070)

    * MINOR: mention KAFKA-15905 in docs "Notable changes in 3.7.1/3.8.0"

    Co-Authored-By: Adrian Preston <prestona@uk.ibm.com>

commit 4f55786
Author: Colin P. McCabe <cmccabe@apache.org>
Date:   Mon May 20 15:41:52 2024 -0700

    KAFKA-16515: Fix the ZK Metadata cache confusion between brokers and controllers

    ZkMetadataCache could theoretically return KRaft controller information from a call to
    ZkMetadataCache.getAliveBrokerNode, which doesn't make sense. KRaft controllers are not part of the
    set of brokers. The only use-case for this functionality was in MetadataCacheControllerNodeProvider
    during ZK migration, where it allowed ZK brokers in migration mode to forward requests to
    kcontrollers when appropriate. This PR changes MetadataCacheControllerNodeProvider to simply
    delegate to quorumControllerNodeProvider in this case.

    Reviewers: José Armando García Sancio <jsancio@apache.org>

commit 90892ae
Author: Colin P. McCabe <cmccabe@apache.org>
Date:   Mon May 20 16:23:27 2024 -0700

    KAFKA-16516: Fix the controller node provider for broker to control channel

    Fix the code in the RaftControllerNodeProvider to query RaftManager to find Node information,
    rather than consulting a static map. Add a RaftManager.voterNode function to supply this
    information. In KRaftClusterTest, add testControllerFailover to get more coverage of controller
    failovers.

    Reviewers: José Armando García Sancio <jsancio@apache.org>

commit 2432a18
Author: KrishVora01 <156789009+KrishVora01@users.noreply.github.com>
Date:   Fri May 24 22:21:02 2024 +0530

    KAFKA-16373: KIP-1028:  Adding code to support Apache Kafka Docker Official Images (apache#16027)

    This PR aims to add JVM based Docker Official Image for Apache Kafka as per the following KIP - https://cwiki.apache.org/confluence/display/KAFKA/KIP-1028%3A+Docker+Official+Image+for+Apache+Kafka

    This PR adds the following functionalities:
    Introduces support for Apache Kafka Docker Official Images via:

    GitHub Workflows:

    - Workflow to prepare static source files for Docker images
    - Workflow to build and test Docker official images
    - Scripts to prepare source files and perform Docker image builds and tests

    A new directory for Docker official images, named docker/docker_official_images. This is the new directory to house all Docker Official Image assets.

    Co-authored-by: Vedarth Sharma <vesharma@confluent.io>

    Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Vedarth Sharma <vesharma@confluent.io>

commit 0143c72
Author: Lianet Magrans <98415067+lianetm@users.noreply.github.com>
Date:   Fri May 24 14:19:43 2024 +0200

    KAFKA-16815: Handle FencedInstanceId in HB response (apache#16047)

    Handle FencedInstanceIdException that a consumer may receive in the heartbeat response. This will be the case when a static consumer is removed from the group by and admin client, and another member joins with the same group.instance.id (allowed in). The first member will receive a FencedInstanceId on its next heartbeat. The expectation is that this should be handled as a fatal error.

    There are no actual changes in logic with this PR, given that without being handled, the FencedInstanceId was being treated as an "unexpected error", which are all treated as fatal errors, so the outcome remains the same. But we're introducing this small change just for accuracy in the logic and the logs: FencedInstanceId is expected during heartbeat, a log line is shown describing the situation and why it happened (and it's treated as a fatal error, just like it was before this PR).

    This PR also improves the test to ensure that the error propagated to the app thread matches the one received in the HB.

    Reviewers: Andrew Schofield <aschofield@confluent.io>, David Jacot <djacot@confluent.io>

commit c5cd190
Author: Gantigmaa Selenge <39860586+tinaselenge@users.noreply.github.com>
Date:   Fri May 24 11:50:47 2024 +0100

    MINOR: Refactor SSL/SASL admin integration tests to not use a custom authorizer (apache#15377)

    Reviewers: Mickael Maison <mickael.maison@gmail.com>

commit 520aa86
Author: Jeff Kim <kimkb2011@gmail.com>
Date:   Fri May 24 03:51:50 2024 -0400

    KAFKA-16626; Lazily convert subscribed topic names to topic ids (apache#15970)

    This patch aims to remove the data structure that stores the conversion from topic names to topic ids which was taking time similar to the actual assignment computation. Instead, we reuse the already existing ConsumerGroupMember.subscribedTopicNames() and do the conversion to topic ids when the iterator is requested.

    Reviewers: David Jacot <djacot@confluent.io>

commit 6941598
Author: Krishna Agarwal <62741600+kagarwal06@users.noreply.github.com>
Date:   Fri May 24 12:16:01 2024 +0530

    KAFKA-16826: Integrate Native Docker Image with github actions (apache#16045)

    This PR integrates the Native docker image with the existing github action jobs for the jvm docker image of AK.

    The integration is done to the following actions:

    docker_build_and_test.yml: Builds the docker image and runs sanity tests and CVE scan
    docker_rc_release.yml: Builds the RC docker image for both amd and arm platform and pushes it to the dockerhub.
    docker_promote.yml: Promotes the RC docker image to the released image tag

    Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Vedarth Sharma <142404391+VedarthConfluent@users.noreply.github.com>

commit de32028
Author: Kuan-Po (Cooper) Tseng <brandboat@gmail.com>
Date:   Fri May 24 05:25:53 2024 +0800

    KAFKA-16828 RackAwareTaskAssignorTest failed (apache#16044)

    Reviewers: Chia-Ping Tsai <chia7712@gmail.com>

commit 11ad5e8
Author: Greg Harris <greg.harris@aiven.io>
Date:   Thu May 23 13:23:18 2024 -0700

    MINOR: Refactor Values class to fix checkstyle, add benchmark, optimize exceptions (apache#15469)

    Signed-off-by: Greg Harris <greg.harris@aiven.io>
    Reviewers: Mickael Maison <mickael.maison@gmail.com>
chiacyu pushed a commit to chiacyu/kafka that referenced this pull request Jun 1, 2024
…he#15970)

This patch aims to remove the data structure that stores the conversion from topic names to topic ids which was taking time similar to the actual assignment computation. Instead, we reuse the already existing ConsumerGroupMember.subscribedTopicNames() and do the conversion to topic ids when the iterator is requested.

Reviewers: David Jacot <djacot@confluent.io>
TaiJuWu pushed a commit to TaiJuWu/kafka that referenced this pull request Jun 8, 2024
…he#15970)

This patch aims to remove the data structure that stores the conversion from topic names to topic ids which was taking time similar to the actual assignment computation. Instead, we reuse the already existing ConsumerGroupMember.subscribedTopicNames() and do the conversion to topic ids when the iterator is requested.

Reviewers: David Jacot <djacot@confluent.io>
gongxuanzhang pushed a commit to gongxuanzhang/kafka that referenced this pull request Jun 12, 2024
…he#15970)

This patch aims to remove the data structure that stores the conversion from topic names to topic ids which was taking time similar to the actual assignment computation. Instead, we reuse the already existing ConsumerGroupMember.subscribedTopicNames() and do the conversion to topic ids when the iterator is requested.

Reviewers: David Jacot <djacot@confluent.io>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
3 participants