Skip to content

KAFKA-15106 fix AbstractStickyAssignor isBalanced predict#13920

Merged
guozhangwang merged 14 commits intoapache:trunkfrom
flashmouse:fix-sticky-assignor-isbalance
Aug 3, 2023
Merged

KAFKA-15106 fix AbstractStickyAssignor isBalanced predict#13920
guozhangwang merged 14 commits intoapache:trunkfrom
flashmouse:fix-sticky-assignor-isbalance

Conversation

@flashmouse
Copy link
Contributor

@flashmouse flashmouse commented Jun 27, 2023

https://issues.apache.org/jira/browse/KAFKA-15106

in 3.5.0 AbstractStickyAssignor may run useless loop in performReassignments because isBalanced have a trivial mistake, and result in rebalance timeout in some situation.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status

@flashmouse
Copy link
Contributor Author

@rajinisivaram @dajac plz have a look, thx!

@divijvaidya
Copy link
Member

Thank you for your first change to Apache Kafka @flashmouse! In the JIRA, you mention that this could be reproduced using a unit test. Can you please add the unit test here which would fail prior to this change and succeed after it?

@flashmouse
Copy link
Contributor Author

@divijvaidya thank you for reply!
the unit test org.apache.kafka.clients.consumer.internals.AbstractStickyAssignorTest#testLargeAssignmentAndGroupWithNonEqualSubscription could use much more time than configured Timeout(90) and this could reproduce easy when partitionCount and consumerCount are large enough and not equal(ex. partitionCount= 200, consumerCount= 20), in this situation isBalanced and performReassignments may run whole loop body.

because both performReassignments and isBalanced in AbstractStickyAssignor are not so efficient(performReassignments is worse), isBalanced could abate performReassignments call times, so fix its logic could speed up this unit test when parameters are small enough.

in my test, after this fix, testLargeAssignmentAndGroupWithNonEqualSubscription could pass when partitionCount= 200, consumerCount = 20, but is still slower than the combination partitionCount= 2000, consumerCount = 2000 because in this test when the 2 parameters are equal, isBalanced satisfy the prediction min >= max - 1 thus could return true immediately.

may be I need add a unit test code with a new commit?

@kirktrue
Copy link
Contributor

@ableegoldman you're pretty familiar with this code, IIUC. If so, could you take a look at this PR? Thanks!

@kirktrue
Copy link
Contributor

@flashmouse I'm a little slow on the uptake, so I'm trying to come up with a scenario. Let's say the following is true:

  • Topic foo has three partitions
  • The consumer group has two consumers, consumer and otherConsumer

In the first case, let's say that consumer has one partition and otherConsumer has two. In that case it's as balanced as possible, right? But the code in isBalanced as written disagrees:

if (consumerPartitionCount < otherConsumerPartitionCount) {
    return false;
}

Because 1 is less than 2 😄 But it's going to run into problems because by returning false from isBalanced, I assume we then trigger a rebalance, which—at best— will cause the ownership of that partition to move from otherConsumer to consumer. When the next balance check for is run, we're in the same position 🤷‍♂️

But with your fix:

if (consumerPartitionCount + 1 < otherConsumerPartitionCount) {
    return false;
}

It's effectively stating that the consumer has to have at least two partitions difference, correct? Because 1 + 1 is not less then 2.

Do I have the above (mostly) correct?

Thanks!

@flashmouse
Copy link
Contributor Author

@kirktrue thank you for reply!

In your case, although isBalanced return false, performReassignments still won't reassign any partition, because it would only do reassign when currentAssignment.get(consumer).size() > currentAssignment.get(otherConsumer).size() + 1, it will just loop all partitions in reassignable list and find no one match it so modified keep false then return done.

I think the problem is isBalanced and performReassignments should have the same predict logic when they judge whether should do reassign. for now performReassignments itself think should do reassign only when 2 consumers that their partition num gap should have at least 2 but isBalanced is 1.

I mean with such situation, I think if we not modify isBalanced, the reassignment logic is still right, but may very slow. performReassignments will do quite a lot unnecessary work, as I said above, it could done finally but cost a long time. you can try unit test testLargeAssignmentAndGroupWithNonEqualSubscription without this pr change, if set partitionCount= 20, consumerCount = 2, it could done with success, but if set partitionCount= 200, consumerCount = 20, it may run very long time. I have run the test and no result after 14min, and it could speed up to 47sec after I apply my change.

Copy link
Contributor

@rreddy-22 rreddy-22 left a comment

Choose a reason for hiding this comment

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

Thanks for the catch! I also observed that the actual reassignments don't take place inside perform reassignments so ideally it should eventually end but only after a long period of time. I ran it for 6hrs and it finally failed with the timeout error. The change would make it faster to determine whether the assignment is balanced or not so I think we should change this irrespective. I think originally the definition of a balanced assignment was that the numbers of topic partitions assigned to consumers differ by at most one but this isn't true for assignments with different subscriptions so I agree we should change it.

@flashmouse
Copy link
Contributor Author

@rreddy-22 that's what I mean, thank you, could any one merge this if possible or help review to verify this pr is useful?

we are now using CooperativeSticky strategy in kafka-client3.5 but meet serious problem that it may trigger rebalance with colossal times(complete one rebalance about 3sec, then keep trigger new balance within more than 10minutes, that's say, at least rebalance 200 times). for now we cannot give much more useful information and we want to verify whether that is caused by this problem after this pr is checked.

@dajac
Copy link
Member

dajac commented Jun 28, 2023

@flashmouse Thanks for the PR. It seems that this condition has been around for a while now. Did you observe a similar issue with version prior to 3.5? I wonder what if changes introduced in 3.5 for the rack aware assignment made this worse than it used to be. I agree that we need to get to the bottom of this.

@flashmouse
Copy link
Contributor Author

@dajac
we haven't seen such many rebalance times before, we before use CooperativeSticky with kafka-client 3.0.0, rebalance times increase very slight.

but we are not use kafka-client 3.5 directly because CooperativeStickyAssignor in 3.5 need upgrad server to 3.5 too if need rack-aware assign logic and we couldn't upgrade server right now so we move AbstractStickyAssignor and CooperativeStickyAssignor logic to our application and move the rackInfo to userData. although we have passd all unit test in kafka-client after do this change, we still doubt whether this change caused rebalance time increase so we're keeping check.

@rreddy-22
Copy link
Contributor

rreddy-22 commented Jun 28, 2023

@flashmouse Thanks for the PR. It seems that this condition has been around for a while now. Did you observe a similar issue with version prior to 3.5? I wonder what if changes introduced in 3.5 for the rack aware assignment made this worse than it used to be. I agree that we need to get to the bottom of this.

The rack awareness changes introduce a whole iteration through all the potential consumers to check if any consumer has a matching rack and in the worst case scenario where no consumer with a matching rack is found, the whole iteration happens all over again. Previously we would only iterate through the list once, so potentially the time could've doubled now.
https://github.com/apache/kafka/blob/3.5/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java#L1389-L1400

@flashmouse
Copy link
Contributor Author

I found another one bug.

In org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor#allSubscriptionsEqual, it would check whether one partition is assigned by multiple consumers in different generation, the logic has 3 if-else body, the last one condition memberGeneration < otherMemberGeneration should add this line below:

allPreviousPartitionsToOwner.put(tp, otherConsumer);

without this line may happen error in such scenario below:
say we have 3 consumers, they own partition1 respectively in generation 1, 2, 3, and then the check sequence is gen3, gen1, gen2, then gen3 and gen2 are all added to consumerToOwnedPartitions so this partition added twice.

this may happen if one partition is owned by at least 3 generation consumer.

@rreddy-22 @kirktrue could u take a look? thx!

otherConsumer, otherMemberGeneration,
tp,
otherMemberGeneration);
allPreviousPartitionsToOwner.put(tp, otherConsumer);
Copy link
Contributor

Choose a reason for hiding this comment

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

So in line 161 we directly added the current consumer to the map allPreviousPartitionsToOwner. And if we choose to keep that line we should remove the put in the first two conditions in lines 175 and 180 and we can keep the change you put.
However, I feel like this would be redundant since we're replacing the otherMember and adding it back if the conditions aren't favorable. My suggestion would be to replace line 161 with get instead of put and then keep the rest of the code as it is.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

updated, plz check again

@philipnee
Copy link
Contributor

Hey @flashmouse - could you explain what do you mean by useless loop?

@flashmouse
Copy link
Contributor Author

Hey @flashmouse - could you explain what do you mean by useless loop?

run while body in performReassignments but couldn't modify assignment

@philipnee
Copy link
Contributor

@flashmouse thansk for the PR - I was the one who wrote the if-else if -else block. I'm a bit surprised that I actually made this mistake, so I held it off for a few days just to make sure. As you have already provided a case that breaks that existing logic, could you implement this as part of the test? Also, could you provide a test case that fails if (consumerPartitionCount < otherConsumerPartitionCount) { but succeeds in if (consumerPartitionCount + 1 < otherConsumerPartitionCount) {

This would help massively for the reviews and documentation.

otherConsumer, otherMemberGeneration,
tp,
memberGeneration);
allPreviousPartitionsToOwner.put(tp, consumer);
Copy link
Contributor

Choose a reason for hiding this comment

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

Does moving this line after the log.warn do anything?

Copy link
Contributor

Choose a reason for hiding this comment

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

Agreed we should move it back to the original position

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

String otherConsumer = allPreviousPartitionsToOwner.put(tp, consumer);
String otherConsumer = allPreviousPartitionsToOwner.get(tp);
if (otherConsumer == null) {
// this partition is not owned by other consumer in the same generation
Copy link
Contributor

Choose a reason for hiding this comment

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

We'd Have to add a put here too

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed, plz take a look

@flashmouse
Copy link
Contributor Author

@flashmouse thansk for the PR - I was the one who wrote the if-else if -else block. I'm a bit surprised that I actually made this mistake, so I held it off for a few days just to make sure. As you have already provided a case that breaks that existing logic, could you implement this as part of the test? Also, could you provide a test case that fails if (consumerPartitionCount < otherConsumerPartitionCount) { but succeeds in if (consumerPartitionCount + 1 < otherConsumerPartitionCount) {

This would help massively for the reviews and documentation.

hi @philipnee thx for reply!

In org.apache.kafka.clients.consumer.internals.AbstractStickyAssignorTest,I add:

  1. testAssignmentAndGroupWithNonEqualSubscriptionNotTimeout, this test may run very long time without fix.
  2. testSubscriptionNotEqualAndAssignSamePartitionWith3Generation, this test should fail without fix.

and please help review #13965 if possible, it mentions another serious bugs in org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.ConstrainedAssignmentBuilder#ConstrainedAssignmentBuilder and may cause rebalance stuck forever.

partitionsPerTopic.put(topic1, partitionInfos(topic1, 1));
int[][] sequence = new int[][]{{1, 2, 3}, {1, 3, 2}, {2, 1, 3}, {2, 3, 1}, {3, 1, 2}, {3, 2, 1}};
for (int[] ints : sequence) {
subscriptions.put(consumer1, buildSubscriptionV2Above(topics(topic), partitions(tp(topic, 0),
Copy link
Contributor

@rreddy-22 rreddy-22 Jul 17, 2023

Choose a reason for hiding this comment

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

Could you please format it in multiple lines like this, it increases readability

Copy link
Contributor

@rreddy-22 rreddy-22 Jul 17, 2023

Choose a reason for hiding this comment

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

    subscriptions.put(
        consumer1,
        buildSubscriptionV2Above(topics(topic1, topic2),
        partitions(tp(topic1, 0)), generationId, 0)
    );

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done, but seems this is enough?

subscriptions.put(consumer1, buildSubscriptionV2Above(topics(topic),
                    partitions(tp(topic, 0), tp(topic, 2)), ints[0], 0));

@philipnee
Copy link
Contributor

Hmm - you can try. But I've had similar, if not the same, failures and I haven't had a chance to examine the cause.

@ableegoldman
Copy link
Member

@flashmouse you can decrease the consumer count. For one thing, this covers the edge case where the consumers do not all have the same topic subscription, which seems to be quite rare. For another, 2,000 consumers is fairly large to begin with. And especially large for a consumer group where the consumers are all given different topic subscriptions.

Honestly it's probably more important to test with a higher partition count than a large group of consumers like this. Are you able to get it passing within 90s with 500 consumers and 5,000 partitions? That feels like a more realistic large-scale test case. If not, fix it with just 500 consumers and maybe run the test with 4,000, then 3,000, then 2,000 partitions, etc, until you can get it to pass.

If it's still not passing with 500 consumers and 2000 partitions, that's a bit troubling. But we can cross that bridge if it comes up

@flashmouse
Copy link
Contributor Author

thx @ableegoldman
I reduce partitions num and increase consumers num, I think all tests influenced by this pr should be PASSED now.

@guozhangwang
Copy link
Contributor

@flashmouse Thanks! I echo other's thoughts that in a unit test, we could refrain ourselves from trying very large scale when trying to validate a specific logic or expose a bug. Just giving a data point here, in StreamsAssignmentScaleTest which is an integration test, given the goal is actually to validate that the assignor is efficient enough such that it can complete the assignment task within a certain time, our scale is around 1000 - 6000 partitions with 2 clients only.

I'm actually wondering if this bug could be exposed with a even small scale, like 3 consumers and 4 partitions at all?

@flashmouse
Copy link
Contributor Author

@flashmouse Thanks! I echo other's thoughts that in a unit test, we could refrain ourselves from trying very large scale when trying to validate a specific logic or expose a bug. Just giving a data point here, in StreamsAssignmentScaleTest which is an integration test, given the goal is actually to validate that the assignor is efficient enough such that it can complete the assignment task within a certain time, our scale is around 1000 - 6000 partitions with 2 clients only.

I'm actually wondering if this bug could be exposed with a even small scale, like 3 consumers and 4 partitions at all?

@guozhangwang thx for reply!

this bug couldn't reproduce in a very small scale, it's actually not a bug, but a trivial mistake would make rebalance logic run very slow. if test scale are too small, it couldn't prove this.

I think the root cause is performReassignments in GeneralAssignmentBuilder is a O(partSize * consumerNum) functions, it is very inefficient.

@guozhangwang
Copy link
Contributor

guozhangwang commented Jul 31, 2023

I think the root cause is performReassignments in GeneralAssignmentBuilder is a O(partSize * consumerNum) functions, it is very inefficient.

Ack, and with your changes, it's now constant and not linear to either num.partitions or num.consumers right?

@flashmouse
Copy link
Contributor Author

I think the root cause is performReassignments in GeneralAssignmentBuilder is a O(partSize * consumerNum) functions, it is very inefficient.

Ack, and with your changes, it's now constant and not linear to either num.partitions or num.consumers right?

I hadn't changed the logic, it is still a slow function. whether execute performReassignments is checked by isBalanced, but it is not effective previously and this pr just fix this.

@guozhangwang
Copy link
Contributor

I hadn't changed the logic, it is still a slow function. whether execute performReassignments is checked by isBalanced, but it is not effective previously and this pr just fix this.

I'm a bit confused now.. my understanding is that with this line of change https://github.com/apache/kafka/pull/13920/files#diff-2814643946ccefef81f480e1280e9b3e3397f4673166ca1ac47db7e3cb6fb760L1175, when num.partitions and num.consumers are not the same, we would no longer run isBalanced | performReassignments once per each combo, and hence it's no longer linear to num.partitions * num.consumers. Of course it would not return immediately (that's only the case when num.partitions and num.consumers happen to be equal), but it would still be much better than the old logic, right?

@flashmouse
Copy link
Contributor Author

I hadn't changed the logic, it is still a slow function. whether execute performReassignments is checked by isBalanced, but it is not effective previously and this pr just fix this.

I'm a bit confused now.. my understanding is that with this line of change https://github.com/apache/kafka/pull/13920/files#diff-2814643946ccefef81f480e1280e9b3e3397f4673166ca1ac47db7e3cb6fb760L1175, when num.partitions and num.consumers are not the same, we would no longer run isBalanced | performReassignments once per each combo, and hence it's no longer linear to num.partitions * num.consumers. Of course it would not return immediately (that's only the case when num.partitions and num.consumers happen to be equal), but it would still be much better than the old logic, right?

ah this is right.

@guozhangwang guozhangwang merged commit e0b7499 into apache:trunk Aug 3, 2023
@guozhangwang
Copy link
Contributor

LGTM. Merged to trunk. Thanks @flashmouse !

@guozhangwang
Copy link
Contributor

Please also update the JIRA ticket status with fix version fields.

@flashmouse
Copy link
Contributor Author

thx, already update jira status, and plz help review #13965 if could, thx.
@guozhangwang @philipnee

@flashmouse flashmouse deleted the fix-sticky-assignor-isbalance branch August 4, 2023 02:21
badaiaqrandista pushed a commit to confluentinc/kafka that referenced this pull request Aug 6, 2023
…e-20230807

* apache-kafka/trunk: (81 commits)
  MINOR: update Kafka Streams state.dir doc (apache#14155)
  KAFKA-15189: only init remote topic metrics when enabled (apache#14133)
  MINOR: improve logging for FK-join (apache#14105)
  KAFKA-15107: Support custom metadata for remote log segment (apache#13984)
  KAFKA-10199: Change to RUNNING if no pending task to recycle exist (apache#14145)
  KAFKA-15106: Fix AbstractStickyAssignor isBalanced predict (apache#13920)
  KAFKA-7438: Replace PowerMockRunner with MockitoJUnitRunner in RetryUtilTest (apache#14143)
  MINOR: Fix debug logs to display TimeIndexOffset (apache#13935)
  KAFKA-12969: Add broker level config synonyms for topic level tiered storage configs (apache#14114)
  KAFKA-15022: [5/N] compute rack aware assignment for standby tasks (apache#14108)
  ...
rreddy-22 added a commit to rreddy-22/kafka-rreddy that referenced this pull request Aug 8, 2023
commit e072706
Author: José Armando García Sancio <jsancio@users.noreply.github.com>
Date:   Tue Aug 8 14:31:42 2023 -0700

    KAFKA-15312; Force channel before atomic file move (apache#14162)

    On ext4 file systems we have seen snapshots with zero-length files. This is possible if
    the file is closed and moved before forcing the channel to write to disk.

    Reviewers: Ron Dagostino <rndgstn@gmail.com>, Alok Thatikunta <athatikunta@confluent.io>

commit a1cb4b4
Author: Lucia Cerchie <luciacerchie@gmail.com>
Date:   Tue Aug 8 12:03:42 2023 -0700

    add changes made before merge (apache#14137)

    Change in response to KIP-941.

    New PR due to merge issue.

    Changes line 57 in the RangeQuery class file from:

    public static <K, V> RangeQuery<K, V> withRange(final K lower, final K upper) {
        return new RangeQuery<>(Optional.of(lower), Optional.of(upper));
    }
    to

    public static <K, V> RangeQuery<K, V> withRange(final K lower, final K upper) {
         return new RangeQuery<>(Optional.ofNullable(lower), Optional.ofNullable(upper));
     }
    Testing strategy:

    Since null values can now be entered in RangeQuerys in order to receive full scans, I changed the logic defining query starting at line 1085 in IQv2StoreIntegrationTest.java from:

            final RangeQuery<Integer, V> query;
            if (lower.isPresent() && upper.isPresent()) {
                query = RangeQuery.withRange(lower.get(), upper.get());
            } else if (lower.isPresent()) {
                query = RangeQuery.withLowerBound(lower.get());
            } else if (upper.isPresent()) {
                query = RangeQuery.withUpperBound(upper.get());
            } else {
                query = RangeQuery.withNoBounds();
            }
    to

    query = RangeQuery.withRange(lower.orElse(null), upper.orElse(null));
    because different combinations of isPresent() in the bounds is no longer necessary.

    Reviewers: John Roesler <vvcephei@apache.org>, Bill Bejeck <bbejeck@apache.org>

commit ff4fed5
Author: Greg Harris <greg.harris@aiven.io>
Date:   Tue Aug 8 10:06:35 2023 -0700

    KAFKA-15031: Add plugin.discovery to Connect worker configuration (KIP-898) (apache#14055)

    Reviewers: Chris Egerton <chrise@aiven.io>

commit 60a5117
Author: Hao Li <1127478+lihaosky@users.noreply.github.com>
Date:   Tue Aug 8 08:01:05 2023 -0700

    KAFKA-15022: [7/N] use RackAwareTaskAssignor in HAAssignor (apache#14139)

    Part of KIP-915.

    - Change TaskAssignor interface to take RackAwareTaskAssignor
    - Integrate RackAwareTaskAssignor to StreamsPartitionAssignor and HighAvailabilityTaskAssignor
    - Update HAAssignor tests

    Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias J. Sax <matthias@confluent.io>

commit 1c04ae8
Author: Matthias J. Sax <matthias@confluent.io>
Date:   Tue Aug 8 07:51:59 2023 -0700

    MINOR: Improve JavaDocs of KafkaStreams `context.commit()` (apache#14163)

    Reviewers: Bill Bejeck <bill@confluent.io>

commit 8dec3e6
Author: Hao Li <1127478+lihaosky@users.noreply.github.com>
Date:   Mon Aug 7 11:21:55 2023 -0700

    KAFKA-15022: [6/N] add rack aware assignor configs and update standby optimizer (apache#14150)

    Part of KIP-925.

    - Add configs for rack aware assignor
    - Update standby optimizer in RackAwareTaskAssignor to have more rounds
    - Refactor some method in RackAwareTaskAssignorTest to test utils so that they can also be used in HighAvailabilityTaskAssignorTest and other tests

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

commit ac6a536
Author: Maros Orsak <maros.orsak159@gmail.com>
Date:   Mon Aug 7 15:19:55 2023 +0200

    MINOR: Fix MiniKdc Java 17 issue in system tests (apache#14011)

    Kafka system tests with Java version 17 are failing on this issue:

    ```python
    TimeoutError("MiniKdc didn't finish startup",)
    Traceback (most recent call last):
      File "/usr/local/lib/python3.6/site-packages/ducktape/tests/runner_client.py", line 186, in _do_run
        data = self.run_test()
      File "/usr/local/lib/python3.6/site-packages/ducktape/tests/runner_client.py", line 246, in run_test
        return self.test_context.function(self.test)
      File "/usr/local/lib/python3.6/site-packages/ducktape/mark/_mark.py", line 433, in wrapper
        return functools.partial(f, *args, **kwargs)(*w_args, **w_kwargs)
      File "/opt/kafka-dev/tests/kafkatest/sanity_checks/test_verifiable_producer.py", line 74, in test_simple_run
        self.kafka.start()
      File "/opt/kafka-dev/tests/kafkatest/services/kafka/kafka.py", line 635, in start
        self.start_minikdc_if_necessary(add_principals)
      File "/opt/kafka-dev/tests/kafkatest/services/kafka/kafka.py", line 596, in start_minikdc_if_necessary
        self.minikdc.start()
      File "/usr/local/lib/python3.6/site-packages/ducktape/services/service.py", line 265, in start
        self.start_node(node, **kwargs)
      File "/opt/kafka-dev/tests/kafkatest/services/security/minikdc.py", line 114, in start_node
        monitor.wait_until("MiniKdc Running", timeout_sec=60, backoff_sec=1, err_msg="MiniKdc didn't finish startup")
      File "/usr/local/lib/python3.6/site-packages/ducktape/cluster/remoteaccount.py", line 754, in wait_until
        allow_fail=True) == 0, **kwargs)
      File "/usr/local/lib/python3.6/site-packages/ducktape/utils/util.py", line 58, in wait_until
        raise TimeoutError(err_msg() if callable(err_msg) else err_msg) from last_exception
    ducktape.errors.TimeoutError: MiniKdc didn't finish startup
    ```

    Specifically, when one runs the test cases and looks at the logs of the MiniKdc:
    ```java
    Exception in thread "main" java.lang.IllegalAccessException: class kafka.security.minikdc.MiniKdc cannot access class sun.security.krb5.Config (in module java.security.jgss) because module java.security.jgss does not export sun.security.krb5 to unnamed module @24959ca4
        at java.base/jdk.internal.reflect.Reflection.newIllegalAccessException(Reflection.java:392)
        at java.base/java.lang.reflect.AccessibleObject.checkAccess(AccessibleObject.java:674)
        at java.base/java.lang.reflect.Method.invoke(Method.java:560)
        at kafka.security.minikdc.MiniKdc.refreshJvmKerberosConfig(MiniKdc.scala:268)
        at kafka.security.minikdc.MiniKdc.initJvmKerberosConfig(MiniKdc.scala:245)
        at kafka.security.minikdc.MiniKdc.start(MiniKdc.scala:123)
        at kafka.security.minikdc.MiniKdc$.start(MiniKdc.scala:375)
        at kafka.security.minikdc.MiniKdc$.main(MiniKdc.scala:366)
        at kafka.security.minikdc.MiniKdc.main(MiniKdc.scala)
    ```

    This error is caused by the fact that sun.security module is no longer supported in Java 16 and higher. Related to the [1].
    There are two ways how to solve it, and I present one of them. The second way is to export the ENV variable during the deployment of the containers using Ducktape in [2].

    [1] - https://openjdk.org/jeps/396
    [2] - https://github.com/apache/kafka/blob/trunk/tests/docker/ducker-ak#L308

    Reviewers: Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>

commit 7a2e11c
Author: Matthias J. Sax <matthias@confluent.io>
Date:   Sun Aug 6 10:20:08 2023 -0700

    MINOR: update Kafka Streams state.dir doc (apache#14155)

    Default state directory was changes in 2.8.0 release (cf KAFKA-10604)

    Reviewers: Guozhang Wang <wangguoz@gmail.com>

commit 748175c
Author: Luke Chen <showuon@gmail.com>
Date:   Sat Aug 5 13:00:16 2023 +0800

    KAFKA-15189: only init remote topic metrics when enabled (apache#14133)

    Only initialize remote topic metrics when system-wise remote storage is enabled to avoid impacting performance for existing brokers. Also add tests.

    Reviewers: Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>

commit faf3635
Author: Matthias J. Sax <matthias@confluent.io>
Date:   Fri Aug 4 21:06:53 2023 -0700

    MINOR: improve logging for FK-join (apache#14105)

    Reviewers: Colt McNealy <colt@littlehorse.io>, Walker Carlson <wcarlson@confluent.io>

commit b3db905
Author: Ivan Yurchenko <ivanyu@aiven.io>
Date:   Fri Aug 4 15:53:25 2023 +0300

    KAFKA-15107: Support custom metadata for remote log segment (apache#13984)

    * KAFKA-15107: Support custom metadata for remote log segment

    This commit does the changes discussed in the KIP-917. Mainly, changes the `RemoteStorageManager` interface in order to return `CustomMetadata` and then ensures these custom metadata are stored, propagated, (de-)serialized correctly along with the standard metadata throughout the whole lifecycle. It introduces the `remote.log.metadata.custom.metadata.max.size` to limit the custom metadata size acceptable by the broker and stop uploading in case a piece of metadata exceeds this limit.

    On testing:
    1. `RemoteLogManagerTest` checks the case when a piece of custom metadata is larger than the configured limit.
    2. `RemoteLogSegmentMetadataTest` checks if `createWithUpdates` works correctly, including custom metadata.
    3. `RemoteLogSegmentMetadataTransformTest`, `RemoteLogSegmentMetadataSnapshotTransformTest`, and `RemoteLogSegmentMetadataUpdateTransformTest` were added to test the corresponding class (de-)serialization, including custom metadata.
    4. `FileBasedRemoteLogMetadataCacheTest` checks if custom metadata are being correctly saved and loaded to a file (indirectly, via `equals`).
    5. `RemoteLogManagerConfigTest` checks if the configuration setting is handled correctly.

    Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Divij Vaidya <diviv@amazon.com>

commit 7782741
Author: Bruno Cadonna <cadonna@apache.org>
Date:   Fri Aug 4 09:07:58 2023 +0200

    KAFKA-10199: Change to RUNNING if no pending task to recycle exist (apache#14145)

    A stream thread should only change to RUNNING if there are no
    active tasks in restoration in the state updater and if there
    are no pending tasks to recycle.

    There are situations in which a stream thread might only have
    standby tasks that are recycled to active task after a rebalance.
    In such situations, the stream thread might be faster in checking
    active tasks in restoration then the state updater removing the
    standby task to recycle from the state updater. If that happens
    the stream thread changes to RUNNING although it should wait until
    the standby tasks are recycled to active tasks and restored.

    Reviewers: Walker Carlson <wcarlson@confluent.io>, Matthias J. Sax <matthias@confluent.io>

commit e0b7499
Author: flashmouse <jackson_666@qq.com>
Date:   Fri Aug 4 02:17:08 2023 +0800

    KAFKA-15106: Fix AbstractStickyAssignor isBalanced predict (apache#13920)

    in 3.5.0 AbstractStickyAssignor may run useless loop in performReassignments  because isBalanced have a trivial mistake, and result in rebalance timeout in some situation.

    Co-authored-by: lixy <lixy@tuya.com>
    Reviewers: Ritika Reddy <rreddy@confluent.io>, Philip Nee <pnee@confluent.io>, Kirk True <kirk@mustardgrain.com>, Guozhang Wang <wangguoz@gmail.com>

commit b9936d6
Author: Yash Mayya <yash.mayya@gmail.com>
Date:   Thu Aug 3 18:07:35 2023 +0100

    KAFKA-7438: Replace PowerMockRunner with MockitoJUnitRunner in RetryUtilTest (apache#14143)

    Reviewers: Chris Egerton <chrise@aiven.io>

commit 7d39d74
Author: Divij Vaidya <diviv@amazon.com>
Date:   Thu Aug 3 11:05:01 2023 +0200

    MINOR: Fix debug logs to display TimeIndexOffset (apache#13935)

    Reviewers: Luke Chen <showuon@gmail.com>

commit d89b26f
Author: Kamal Chandraprakash <kchandraprakash@uber.com>
Date:   Thu Aug 3 13:56:00 2023 +0530

    KAFKA-12969: Add broker level config synonyms for topic level tiered storage configs (apache#14114)

    KAFKA-12969: Add broker level config synonyms for topic level tiered storage configs.

    Topic -> Broker Synonym:
    local.retention.bytes -> log.local.retention.bytes
    local.retention.ms -> log.local.retention.ms

    We cannot add synonym for `remote.storage.enable` topic property as it depends on KIP-950

    Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>

commit bb48b15
Author: Hao Li <1127478+lihaosky@users.noreply.github.com>
Date:   Wed Aug 2 19:20:23 2023 -0700

    KAFKA-15022: [5/N] compute rack aware assignment for standby tasks (apache#14108)

    Part of KIP-925.

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

commit 8aaf7da
Author: Abhijeet Kumar <abhijeet.cse.kgp@gmail.com>
Date:   Wed Aug 2 12:27:25 2023 +0530

    KAFKA-15236: Rename tiered storage metrics (apache#14074)

    Rename tiered storage metrics

    Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>

commit ffe5f9f
Author: Kamal Chandraprakash <kchandraprakash@uber.com>
Date:   Wed Aug 2 12:05:40 2023 +0530

    KAFKA-15272: Fix the logic which finds candidate log segments to upload it to tiered storage (apache#14128)

    In tiered storage, a segment is eligible for deletion from local disk when it gets uploaded to the remote storage.

    If the topic active segment contains some messages and there are no new incoming messages, then the active segment gets rotated to passive segment after the configured log.roll.ms timeout.

    The logic to find the candidate segment in RemoteLogManager does not include the recently rotated passive segment as eligible to upload it to remote storage so the passive segment won't be removed even after if it breaches by retention time/size. (ie) Topic won't be empty after it becomes stale.

    Added unit test to cover the scenario which will fail without this patch.

    Reviewers: Christo Lolov <lolovc@amazon.com>, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>

commit 0ce1640
Author: Hao Li <1127478+lihaosky@users.noreply.github.com>
Date:   Tue Aug 1 17:33:24 2023 -0700

    KAFKA-15022: [4/N] use client tag assignor for rack aware standby task assignment (apache#14097)

    Part of KIP-925.

    For rack aware standby task assignment, we can either use the already existing "rack tags" or as a fall-back the newly added "rack.id". This PR unifies both without the need to change the actual standby task assignment logic.

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

commit b9a4554
Author: Greg Harris <greg.harris@aiven.io>
Date:   Tue Aug 1 10:05:46 2023 -0700

    KAFKA-15244: Remove PluginType.from(Class) (apache#14089)

    Reviewers: Chris Egerton <chrise@aiven.io>

commit 7ecf518
Author: Christo Lolov <lolovc@amazon.com>
Date:   Tue Aug 1 15:10:39 2023 +0100

    KAFKA-14661: Upgrade Zookeeper to 3.8.1 (apache#13260)

    Reviewers: Divij Vaidya <diviv@amazon.com>, Mickael Maison <mickael.maison@gmail.com>

commit 660e6fe
Author: hzh0425 <642256541@qq.com>
Date:   Tue Aug 1 14:53:42 2023 +0800

    MINOR: Fix some typos in remote.metadata.storage (apache#13133)

    Fix some typos in remote.metadata.storage

    Reviewers: Luke Chen <showuon@gmail.com>

commit 938fee2
Author: David Arthur <mumrah@gmail.com>
Date:   Mon Jul 31 09:21:22 2023 -0400

    Fix a Scala 2.12 compile issue (apache#14126)

    Reviewers: Luke Chen <showuon@gmail.com>, Qichao Chu

commit 3ba718e
Author: Yash Mayya <yash.mayya@gmail.com>
Date:   Fri Jul 28 19:35:42 2023 +0100

    MINOR: Remove duplicate instantiation of MockConnectMetrics in AbstractWorkerSourceTaskTest (apache#14091)

    Reviewers: Christo Lolov <christololov@gmail.com>, Manyanda Chitimbo <manyanda.chitimbo@gmail.com>, Greg Harris <greg.harris@aiven.io>

commit 1574b9f
Author: David Jacot <djacot@confluent.io>
Date:   Fri Jul 28 20:28:54 2023 +0200

    MINOR: Code cleanups in group-coordinator module (apache#14117)

    This patch does a few code cleanups in the group-coordinator module.

    It renames Coordinator to CoordinatorShard;
    It renames ReplicatedGroupCoordinator to GroupCoordinatorShard. I was never really happy with this name. The new name makes more sense to me;
    It removes TopicPartition from the GroupMetadataManager. It was only used in log messages. The log context already includes it so we don't have to log it again.
    It renames assignors to consumerGroupAssignors.

    Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>

commit 3709901
Author: Ritika Reddy <98577846+rreddy-22@users.noreply.github.com>
Date:   Fri Jul 28 10:30:04 2023 -0700

    KAFKA-14702: Extend server side assignor to support rack aware replica placement (apache#14099)

    This patch updates the `PartitionAssignor` interface to support rack-awareness. The change introduces the `SubscribedTopicDescriber` interface that can be used to retrieve topic metadata such as the number of partitions or the racks from within an assignor. We use an interface because it allows us to wrap internal data structures instead of having to copy them. It is more efficient.

    Reviewers: David Jacot <djacot@confluent.io>

commit 32c39c8
Author: David Arthur <mumrah@gmail.com>
Date:   Fri Jul 28 13:02:47 2023 -0400

    KAFKA-15263 Check KRaftMigrationDriver state in each event (apache#14115)

    Reviewers: Colin P. McCabe <cmccabe@apache.org>

commit 811ae01
Author: Philip Nee <pnee@confluent.io>
Date:   Fri Jul 28 09:11:20 2023 -0700

    MINOR: Test assign() and assignment() in the integration test (apache#14086)

    A missing piece from KAFKA-14950. This is to test assign() and assignment() in the integration test.

    Also fixed an accidental mistake in the committed API.

    Reviewers: Jun Rao <junrao@gmail.com>

commit 19f9e1e
Author: Jeff Kim <kimkb2011@gmail.com>
Date:   Fri Jul 28 09:13:27 2023 -0400

    KAFKA-14501: Implement Heartbeat protocol in new GroupCoordinator (apache#14056)

    This patch implements the existing Heartbeat API in the new Group Coordinator.

    Reviewers: David Jacot <djacot@confluent.io>

commit dcabc29
Author: David Jacot <djacot@confluent.io>
Date:   Fri Jul 28 14:49:48 2023 +0200

    KAFKA-14048; CoordinatorContext should be protected by a lock (apache#14090)

    Accessing the `CoordinatorContext` in the `CoordinatorRuntime` should be protected by a lock. The runtime guarantees that the context is never access concurrently however it is accessed by multiple threads. The lock is here to ensure that we have a proper memory barrier. The patch does the following:
    1) Adds a lock to `CoordinatorContext`;
    2) Adds helper methods to get the context and acquire/release the lock.
    3) Allow transition from Failed to Loading. Previously, the context was recreated in this case.

    Reviewers: Justine Olshan <jolshan@confluent.io>

commit afe631c
Author: James Shaw <js102@zepler.net>
Date:   Fri Jul 28 10:45:15 2023 +0100

    KAFKA-14967: fix NPE in CreateTopicsResult in MockAdminClient (apache#13671)

    Co-authored-by: James Shaw <james.shaw@masabi.com>
    Reviewers: Mickael Maison <mickael.maison@gmail.com>

commit 722b259
Author: Christo Lolov <lolovc@amazon.com>
Date:   Fri Jul 28 06:40:37 2023 +0100

    KAFKA-14038: Optimise calculation of size for log in remote tier (apache#14049)

    Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>

commit 10bcd4f
Author: Colin Patrick McCabe <cmccabe@apache.org>
Date:   Thu Jul 27 17:01:55 2023 -0700

    KAFKA-15213: provide the exact offset to QuorumController.replay (apache#13643)

    Provide the exact record offset to QuorumController.replay() in all cases. There are several situations
    where this is useful, such as logging, implementing metadata transactions, or handling broker
    registration records.

    In the case where the QC is inactive, and simply replaying records, it is easy to compute the exact
    record offset from the batch base offset and the record index.

    The active QC case is more difficult. Technically, when we submit records to the Raft layer, it can
    choose a batch base offset later than the one we expect, if someone else is also adding records.
    While the QC is the only entity submitting data records, control records may be added at any time.
    In the current implementation, these are really only used for leadership elections. However, this
    could change with the addition of quorum reconfiguration or similar features.

    Therefore, this PR allows the QC to tell the Raft layer that a record append should fail if it
    would have resulted in a batch base offset other than what was expected. This in turn will trigger a
    controller failover. In the future, if automatically added control records become more common, we
    may wish to have a more sophisticated system than this simple optimistic concurrency mechanism. But
    for now, this will allow us to rely on the offset as correct.

    In order that the active QC can learn what offset to start writing at, the PR also adds a new
    RaftClient#endOffset function.

    At the Raft level, this PR adds a new exception, UnexpectedBaseOffsetException. This gets thrown
    when we request a base offset that doesn't match the one the Raft layer would have given us.
    Although this exception should cause a failover, it should not be considered a fault. This
    complicated the exception handling a bit and motivated splitting more of it out into the new
    EventHandlerExceptionInfo class. This will also let us unit test things like slf4j log messages a
    bit better.

    Reviewers: David Arthur <mumrah@gmail.com>, José Armando García Sancio <jsancio@apache.org>

commit e5861ee
Author: Alyssa Huang <ahuang@confluent.io>
Date:   Thu Jul 27 13:12:25 2023 -0700

    [MINOR] Add latest versions to kraft upgrade kafkatest (apache#14084)

    Reviewers: Ron Dagostino <rndgstn@gmail.com>

commit 6f39ef0
Author: Justine Olshan <jolshan@confluent.io>
Date:   Thu Jul 27 09:36:32 2023 -0700

    MINOR: Adjust Invalid Record Exception for Invalid Txn State as mentioned in KIP-890 (apache#14088)

    Invalid record is a newer error. INVALID_TXN_STATE has been around as long as transactions and is not retriable. This is the desired behavior.

commit 29825ee
Author: David Jacot <djacot@confluent.io>
Date:   Thu Jul 27 13:18:10 2023 +0200

    KAFKA-14499: [3/N] Implement OffsetCommit API (apache#14067)

    This patch introduces the `OffsetMetadataManager` and implements the `OffsetCommit` API for both the old rebalance protocol and the new rebalance protocol. It introduces version 9 of the API but keeps it as unstable for now. The patch adds unit tests to test the API. Integration tests will be done separately.

    Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>

commit 353141e
Author: Divij Vaidya <diviv@amazon.com>
Date:   Thu Jul 27 12:33:34 2023 +0200

    KAFKA-15251: Add 3.5.1 to system tests (apache#14069)

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

commit d2fc907
Author: Jeff Kim <kimkb2011@gmail.com>
Date:   Thu Jul 27 02:02:29 2023 -0400

    KAFKA-14500; [6/6] Implement SyncGroup protocol in new GroupCoordinator (apache#14017)

    This patch implements the SyncGroup API in the new group coordinator. All the new unit tests are based on the existing scala tests.

    Reviewers: David Jacot <djacot@confluent.io>

commit ed44bcd
Author: Hao Li <1127478+lihaosky@users.noreply.github.com>
Date:   Wed Jul 26 16:02:52 2023 -0700

    KAFKA-15022: [3/N] use graph to compute rack aware assignment for active stateful tasks (apache#14030)

    Part of KIP-925.

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

commit 8135b6d
Author: Said Boudjelda <bmscomp@gmail.com>
Date:   Wed Jul 26 19:52:02 2023 +0200

    KAFKA-15235: Fix broken coverage reports since migration to Gradle 8.x (apache#14075)

    Reviewers: Divij Vaidya <diviv@amazon.com>

commit e5fb9b6
Author: Said Boudjelda <bmscomp@gmail.com>
Date:   Wed Jul 26 19:12:27 2023 +0200

    MINOR: upgrade version of gradle plugin (ben-manes.versions) to 0.47.0 (apache#14098)

    Reviewers: Divij Vaidya <diviv@amazon.com>

commit a900794
Author: David Arthur <mumrah@gmail.com>
Date:   Wed Jul 26 12:54:59 2023 -0400

    KAFKA-15196 Additional ZK migration metrics (apache#14028)

    This patch adds several metrics defined in KIP-866:

    * MigratingZkBrokerCount: the number of zk brokers registered with KRaft
    * ZkWriteDeltaTimeMs: time spent writing MetadataDelta to ZK
    * ZkWriteSnapshotTimeMs: time spent writing MetadataImage to ZK
    * Adds value 4 for "ZK" to ZkMigrationState

    Also fixes a typo in the metric name introduced in apache#14009 (ZKWriteBehindLag -> ZkWriteBehindLag)

    Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>

commit 6d81698
Author: sciclon2 <74413315+sciclon2@users.noreply.github.com>
Date:   Wed Jul 26 15:48:09 2023 +0200

    KAFKA-15243: Set decoded user names to DescribeUserScramCredentialsResponse (apache#14094)

    Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>

commit ff390ab
Author: vamossagar12 <sagarmeansocean@gmail.com>
Date:   Wed Jul 26 17:56:20 2023 +0530

    [MINOR] Fix Javadoc comment in KafkaFuture#toCompletionStage (apache#14100)

    Fix Javadoc comment in KafkaFuture#toCompletionStage

    Reviewers: Luke Chen <showuon@gmail.com>

commit bb677c4
Author: Federico Valeri <fedevaleri@gmail.com>
Date:   Wed Jul 26 12:04:34 2023 +0200

    KAFKA-14583: Move ReplicaVerificationTool to tools (apache#14059)

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

commit 4d30cbf
Author: Said Boudjelda <bmscomp@gmail.com>
Date:   Wed Jul 26 11:21:36 2023 +0200

    MINOR: Upgrade the minor version of snappy dependency to 1.1.10.3 (apache#14072)

    Reviewers: Divij Vaidya <diviv@amazon.com>

commit 206a4af
Author: Divij Vaidya <diviv@amazon.com>
Date:   Wed Jul 26 11:19:56 2023 +0200

    MINOR: Add co-authors to release email template (apache#14080)

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

commit 46a8a28
Author: vamossagar12 <sagarmeansocean@gmail.com>
Date:   Wed Jul 26 07:21:23 2023 +0530

    KAFKA-15218: Avoid NPE thrown while deleting topic and fetch from follower concurrently (apache#14051)

    When deleting topics, we'll first clear all the remoteReplicaMap when stopPartitions here. But this time, there might be fetch request coming from follower, and try to check if the replica is eligible to be added into ISR here. At this moment, NPE will be thrown. Although it's fine since this topic is already deleted, it'd be better to avoid it happen.

    Reviewers: Luke Chen <showuon@gmail.com>

commit af1f50f
Author: Matthias J. Sax <matthias@confluent.io>
Date:   Tue Jul 25 14:56:58 2023 -0700

    MINOR: fix docs markup (apache#14085)

    Reviewers: Qichao Chu (@ex172000), Mickael Maison <mickael.maison@gmail.com>

commit e794bc7
Author: David Arthur <mumrah@gmail.com>
Date:   Tue Jul 25 16:05:04 2023 -0400

    MINOR: Add a Builder for KRaftMigrationDriver (apache#14062)

    Reviewers: Justine Olshan <jolshan@confluent.io>

commit 8b027b6
Author: tison <wander4096@gmail.com>
Date:   Tue Jul 25 23:56:49 2023 +0800

    MINOR: Fix typo in ProduceRequest.json (apache#14070)

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

commit 08b3820
Author: Yash Mayya <yash.mayya@gmail.com>
Date:   Tue Jul 25 14:03:29 2023 +0100

    KAFKA-15238: Move DLQ reporter setup from the DistributedHerder's tick thread to the sink task thread (apache#14079)

    Reviewers: Chris Egerton <chrise@aiven.io>

commit 58b8c5c
Author: Chris Egerton <chrise@aiven.io>
Date:   Tue Jul 25 05:12:46 2023 -0700

    MINOR: Downgrade log level for conflicting Connect plugin aliases (apache#14081)

    Reviewers: Greg Harris <greg.harris@aiven.io>

commit c7de30f
Author: Colin Patrick McCabe <cmccabe@apache.org>
Date:   Mon Jul 24 21:13:58 2023 -0700

    KAFKA-15183: Add more controller, loader, snapshot emitter metrics (apache#14010)

    Implement some of the metrics from KIP-938: Add more metrics for
    measuring KRaft performance.

    Add these metrics to QuorumControllerMetrics:
        kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount
        kafka.controller:type=KafkaController,name=EventQueueOperationsStartedCount
        kafka.controller:type=KafkaController,name=EventQueueOperationsTimedOutCount
        kafka.controller:type=KafkaController,name=NewActiveControllersCount

    Create LoaderMetrics with these new metrics:
        kafka.server:type=MetadataLoader,name=CurrentMetadataVersion
        kafka.server:type=MetadataLoader,name=HandleLoadSnapshotCount

    Create SnapshotEmitterMetrics with these new metrics:
        kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedBytes
        kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs

    Reviewers: Ron Dagostino <rndgstn@gmail.com>

commit 79b8c96
Author: David Mao <47232755+splett2@users.noreply.github.com>
Date:   Mon Jul 24 13:22:25 2023 -0700

    KAFKA-14990: Dynamic producer ID expiration should be applied on a broker restart (apache#13707)

    Dynamic overrides for the producer ID expiration config are not picked up on broker restart in Zookeeper mode. Based on the integration test, this does not apply to KRaft mode.

    Adds a broker restart that fails without the corresponding KafkaConfig change.

    Reviewers: Justine Olshan <jolshan@confluent.io>

commit 38781f9
Author: Justine Olshan <jolshan@confluent.io>
Date:   Mon Jul 24 13:08:57 2023 -0700

    KAFKA-14920: Address timeouts and out of order sequences (apache#14033)

    When creating a verification state entry, we also store sequence and epoch. On subsequent requests, we will take the latest epoch seen and the earliest sequence seen. That way, if we try to append a sequence after the earliest seen sequence, we can block that and retry. This addresses potential OutOfOrderSequence loops caused by errors during verification (coordinator loading, timeouts, etc).

    Reviewers:  David Jacot <david.jacot@gmail.com>,  Artem Livshits <alivshits@confluent.io>
jeqo pushed a commit to aiven/kafka that referenced this pull request Aug 15, 2023
)

in 3.5.0 AbstractStickyAssignor may run useless loop in performReassignments  because isBalanced have a trivial mistake, and result in rebalance timeout in some situation.

Co-authored-by: lixy <lixy@tuya.com>
Reviewers: Ritika Reddy <rreddy@confluent.io>, Philip Nee <pnee@confluent.io>, Kirk True <kirk@mustardgrain.com>, Guozhang Wang <wangguoz@gmail.com>
jeqo pushed a commit to jeqo/kafka that referenced this pull request Aug 15, 2023
)

in 3.5.0 AbstractStickyAssignor may run useless loop in performReassignments  because isBalanced have a trivial mistake, and result in rebalance timeout in some situation.

Co-authored-by: lixy <lixy@tuya.com>
Reviewers: Ritika Reddy <rreddy@confluent.io>, Philip Nee <pnee@confluent.io>, Kirk True <kirk@mustardgrain.com>, Guozhang Wang <wangguoz@gmail.com>
jeqo pushed a commit to jeqo/kafka that referenced this pull request Aug 15, 2023
)

in 3.5.0 AbstractStickyAssignor may run useless loop in performReassignments  because isBalanced have a trivial mistake, and result in rebalance timeout in some situation.

Co-authored-by: lixy <lixy@tuya.com>
Reviewers: Ritika Reddy <rreddy@confluent.io>, Philip Nee <pnee@confluent.io>, Kirk True <kirk@mustardgrain.com>, Guozhang Wang <wangguoz@gmail.com>
rreddy-22 pushed a commit to rreddy-22/kafka-rreddy that referenced this pull request Sep 20, 2023
)

in 3.5.0 AbstractStickyAssignor may run useless loop in performReassignments  because isBalanced have a trivial mistake, and result in rebalance timeout in some situation.

Co-authored-by: lixy <lixy@tuya.com>
Reviewers: Ritika Reddy <rreddy@confluent.io>, Philip Nee <pnee@confluent.io>, Kirk True <kirk@mustardgrain.com>, Guozhang Wang <wangguoz@gmail.com>
showuon pushed a commit that referenced this pull request Oct 20, 2023
in 3.5.0 AbstractStickyAssignor may run useless loop in performReassignments  because isBalanced have a trivial mistake, and result in rebalance timeout in some situation.

Co-authored-by: lixy <lixy@tuya.com>
Reviewers: Ritika Reddy <rreddy@confluent.io>, Philip Nee <pnee@confluent.io>, Kirk True <kirk@mustardgrain.com>, Guozhang Wang <wangguoz@gmail.com>
@showuon
Copy link
Member

showuon commented Oct 20, 2023

Backported to 3.5 branch.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

9 participants