Task pairs is an optimization that is enabled in the current sticky task
assignor.
The basic idea is that every time we add a task A to a client that has
tasks B, C, we add pairs (A, B) and (A, C) to a global collection of
pairs. When adding a standby task, we then prioritize creating standby
tasks that create new task pairs. If this does not work, we fall back to
the original behavior.
The complexity of this optimization is fairly significant, and its
usefulness is questionable, the HighAvailabilityAssignor does not seem
to have such an optimization, and the absence of this optimization does
not seem to have caused any problems that I know of. I could not find
any what this optimization is actually trying to achieve.
A side effect of it is that we will sometimes avoid “small loops”, such
as
Node A: ActiveTask1, StandbyTask2 Node B: ActiveTask2,
StandbyTask1 Node C: ActiveTask3,
StandbyTask4 Node D: ActiveTask4,
StandbyTask3
So a small loop like this, worst case losing two nodes will cause 2
tasks to go down, so the assignor is preferring
Node A: ActiveTask1, StandbyTask4 Node B: ActiveTask2,
StandbyTask1 Node C: ActiveTask3,
StandbyTask2 Node D: ActiveTask4,
StandbyTask3
Which is a “big loop” assignment, where worst-case losing two nodes will
cause at most 1 task to be unavailable. However, this optimization seems
fairly niche, and also the current implementation does not seem to
implement it in a direct form, but a more relaxed constraint which
usually, does not always avoid small loops. So it remains unclear
whether this is really the intention behind the optimization. The
current unit tests of the StickyTaskAssignor pass even after removing
the optimization.
The pairs optimization has a worst-case quadratic space and time
complexity in the number of tasks, and make a lot of other optimizations
impossible, so I’d suggest we remove it. I don’t think, in its current
form, it is suitable to be implemented in a broker-side assignor. Note,
however, if we identify a useful effect of the code in the future, we
can work on finding an efficient algorithm that can bring the
optimization to our broker-side assignor.
This reduces the runtime of our worst case benchmark by 10x.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
* While creating share group init requests in
`GroupMetadataManager.shareGroupHeartbeat`, we check for topics in
`initializing` state and if they are a certain amount of time old, we
issue retry requests for the same.
* The interval for considering initializing topics as old was based of
`offsetsCommitTimeoutMs` and was not configurable.
* In this PR, we remedy the situation by introducing a new config to
supply the value. The default is `30_000` which is a
heuristic based on the fact that the share coordinator `persister`
retries request with exponential backoff, with upper cap of `30_000`
seconds.
* Tests have been updated wherever applicable.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Lan Ding
<isDing_L@163.com>, TaiJuWu <tjwu1217@gmail.com>, Andrew Schofield
<aschofield@confluent.io>
This is a very mechanical and obvious change that is making most
accessors in ProcessState constant time O(1), instead of linear time
O(n), by computing the collections and aggregations at insertion time,
instead of every time the value is accessed.
Since the accessors are used in deeply nested loops, this reduces the
runtime of our worst case benchmarks by ~14x.
Reviewers: Bill Bejeck <bbejeck@apache.org>
When the group coordinator is processing a heartbeat from a share
consumer, it must decide whether the recompute the assignment. Part of
this decision hinges on whether the assigned partitions match the
partitions initialised by the share coordinator. However, when the set
of subscribed topics changes, there may be initialised partitions which
are not currently assigned. Topics which are not subscribed should be
omitted from the calculation about whether to recompute the assignment.
Co-authored-by: Sushant Mahajan <smahajan@confluent.io>
Reviewers: Lan Ding <53332773+DL1231@users.noreply.github.com>, Sushant
Mahajan <smahajan@confluent.io>, Apoorv Mittal
<apoorvmittal10@gmail.com>
Finalise the share group SimpleAssignor for heterogeneous subscriptions.
The assignor code is much more accurate about the number of partitions
assigned to each member, and the number of members assigned for each
partition. It eliminates the idea of hash-based assignment because that
has been shown to the unhelpful. The revised code is very much more
effective at assigning evenly as the number of members grows and shrinks
over time.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
* There are instances where share group delete calls in group
coordinator (`onPartitionsDelete`, `deleteShareGroups`) where we lookup
the metadata image to fetch the topic id/partitions/topic name for a
topic name/id. However, there have been
instances where the looked up info was not found due to cluster being
under load or the underlying topic being deleted and information not
propagated correctly.
* To remedy the same, this PR adds checks to determine that topic is
indeed present in the image before the lookups thus preventing NPEs. The
problematic situations are logged.
* New tests have been added for `GroupMetadataManger` and
`GroupCoordinatorService`.
Reviewers: Andrew Schofield <aschofield@confluent.io>
While testing the code in https://github.com/apache/kafka/pull/19820, it
became clear that the error handling problems were due to the underlying
Admin API. This PR fixes the error handling for top-level errors in the
AlterShareGroupOffsets RPC.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Lan Ding
<isDing_L@163.com>, TaiJuWu <tjwu1217@gmail.com>
* When a `ShareGroup*` record is replayed in group
metadata manager, there is a call to check if the group exists. If the
group does not exist - we are throwing an exception which is
unnecessary.
* In this PR, we have added check to ignore this exception.
* New test to validate the logic has been added.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Dongnuo Lyu
<139248811+dongnuo123@users.noreply.github.com>
Cleanup hasMemberSubscriptionChanged.
Remove unused InvalidRegularExpressionException from the signature.
Reviewers: Sean Quah <squah@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>, TengYao Chi <frankvicky@apache.org>
The OffsetFetch API does not support top level errors in version 1.
Hence, the top level error must be returned at the partition level.
Side note: It is a tad annoying that we create error response in
multiple places (e.g. KafkaApis, Group CoordinatorService). There were a
reason for this but I cannot remember.
Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Sean Quah <squah@confluent.io>, Ken Huang <s7133700@gmail.com>, TengYao Chi <frankvicky@apache.org>
KIP-1071 does not currently support all features planned in the KIP. We
should reject any requests that are using features that are currently
not implemented.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Matthias J. Sax
<matthias@confluent.io>, Bill Bejeck <bill@confluent.io>
ConsumerGroupDescribe with an empty group id returns a response containing `null` groupId in a non-nullable field. Since the response cannot be serialized, this results in UNKNOWN_SERVER_ERROR being returned to the client. This PR sets the group id in the response to an empty string instead and adds request tests for empty group id.
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
* The share group rebalance metric was not being invoked at the
appropriate group id bump position.
* This PR solves the issue.
* The metric name has been updated
(s/rebalance-rate/share-group-rebalance-rate,
s/rebalance-count/share-group-rebalance-count/)
* Updated tests in `GroupMetadataManagerTest` and
`GroupCoordinatorMetricsTest`
Reviewers: Andrew Schofield <aschofield@confluent.io>
Finalise the share group SimpleAssignor for homogeneous subscriptions.
The assignor code is much more accurate about the number of partitions
assigned to each member, and the number of members assigned for each
partition. It eliminates the idea of hash-based assignment because that
has been shown to the unhelpful. The revised code is very much more
effective at assigning evenly as the number of members grows and shrinks
over time.
A future PR will address the code for heterogeneous subscriptions.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
Consumers can subscribe to an RE2J SubscriptionPattern that will be
resolved and maintained on the server-side (KIP-848). Currently, those
regexes are refreshed on the coordinator when a consumer subscribes to a
new regex, or if there is a new topic metadata image (to ensure regex
resolution stays up-to-date with existing topics)
But with
[KAFKA-18813](https://issues.apache.org/jira/browse/KAFKA-18813), the
topics matching a regex are filtered based on ACLs. This generates a new
situation, as regexes resolution do not stay up-to-date as topics become
visible (ACLs added/delete).
This patch introduces time-based refresh for the subscribed regex by
- Adding internal `group.consumer.regex.batch.refresh.max.interval.ms`
config
that controls the refresh interval.
- Schedule a regex refresh when updating regex subscription if the
latest refresh is older than the max interval.
Reviewers: David Jacot <djacot@confluent.io>
Simplify Set initialization and reduce the overhead of creating extra
collections.
The changes mostly include:
- new HashSet<>(List.of(...))
- new HashSet<>(Arrays.asList(...)) / new HashSet<>(asList(...))
- new HashSet<>(Collections.singletonList()) / new
HashSet<>(singletonList())
- new HashSet<>(Collections.emptyList())
- new HashSet<>(Set.of())
This change takes the following into account, and we will not change to
Set.of in these scenarios:
- Require `mutability` (UnsupportedOperationException).
- Allow `duplicate` elements (IllegalArgumentException).
- Allow `null` elements (NullPointerException).
- Depend on `Ordering`. `Set.of` does not guarantee order, so it could
make tests flaky or break public interfaces.
Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
* Add `group.share.assignors` config to `GroupCoordinatorConfig`.
* Send `rackId` in share group heartbeat request if it's not null.
* Add integration test `testShareConsumerWithRackAwareAssignor`.
Reviewers: Lan Ding <53332773+DL1231@users.noreply.github.com>, Andrew
Schofield <aschofield@confluent.io>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
The mapKey optimisation can be used in some KIP-932 RPC schemas to
improve efficiency of some key-based accesses.
* AlterShareGroupOffsetsResponse
* ShareFetchRequest
* ShareFetchResponse
* ShareAcknowledgeRequest
* ShareAcknowledgeResponse
Reviewers: Andrew Schofield <aschofield@confluent.io>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
We should behave more like a consumer group and make sure to not be
subscribed to the input topics anymore when the last member leaves the
group. We don't do this right now because our topology is still
initialized even after the last member leaves the group.
This will allow:
* Offsets to expire and be cleaned up.
* Offsets to be deleted through admin API calls.
Reviewers: Bill Bejeck <bbejeck@apache.org>
A heartbeat might be sent to the group coordinator, claiming to own
tasks that we do not know about. We need some logic to handle those
requests. In KIP-1071, we propose to return `INVALID_REQUEST` error
whenever this happens, effectively letting the clients crash.
This behavior will, however, make topology updates impossible. Bruno
Cadonna proposed to only check that owned tasks match our set of
expected tasks if the topology epochs between the group and the client
match. The aim of this change is to implement a check and a behavior
for the first version of the protocol, which is to always return
`INVALID_REQUEST` if an unknown task is sent to the group coordinator.
We can relax this constraint once we allow topology updating with
topology epochs.
To efficiently check this whenever we receive a heartbeat containing
tasks, we precompute the number of tasks for each subtopology. This also
benefits the performance of the assignor.
Reviewers: Bill Bejeck <bbejeck@apache.org>
* Use metadata hash to replace subscription metadata.
* Remove `ShareGroupPartitionMetadataKey` and
`ShareGroupPartitionMetadataValue`.
* Use `subscriptionTopicNames` and `metadataImage` to replace
`subscriptionMetadata` in `subscribedTopicsChangeMap` function.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot
<djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
This is the initial documentation for KIP-932 preview in AK 4.1. The aim
is to get very minimal docs in before the cutoff. Longer term, more
comprehensive documentation will be provided for AK 4.2.
The PR includes:
* Generation of group-level configuration documentation
* Add link to KafkaShareConsumer to API docs
* Add a summary of share group rational to design docs
* Add basic operations information for share groups to ops docs
* Add upgrade note describing arrival of KIP-932 preview in 4.1
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
---------
Co-authored-by: Apoorv Mittal <apoorvmittal10@gmail.com>
* Use metadata hash to replace subscription metadata.
* Remove `StreamsGroupPartitionMetadataKey` and
`StreamsGroupPartitionMetadataValue`.
* Check whether `configuredTopology` is empty. If it's, call
`InternalTopicManager.configureTopics` and set the result to the group.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
PR streamlines the logs when delete share group or offset is triggered.
Also fixes the response when group is not found while deleting share
group.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Sushant Mahajan
<smahajan@confluent.io>
Minor updates to RangeSet: * Disallow ranges with negative size *
Disallow ranges with more than Integer.MAX_VALUE elements * Fix
equals() so that all empty RangeSets are equal, to follow the Set
interface definition better. * Reimplement hashCode() to follow the
Set interface definition.
Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
- A new method `assignablePartitions` was added to the
`SubscribedTopicDescriber`in https://github.com/apache/kafka/pull/19026.
This method was required for computing assignments for share groups
(KIP-932).
- However, since the describer is a public interface and is used to
encapsulate methods which return all subscribed partitions (KIP-848),
`assignablePartitions` is deemed inconsistent with this interface.
- Hence, this PR extends the `GroupSpec` interface to add a method
`isPartitionAssignable` which will serve the same purpose. The
`assignablePartitions` has been removed from the describer.
- Tests have been updated for the assigners and spec and removed from
describer as required.
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Jacot
<djacot@confluent.io>
* Add `topicHashCache` to `GroupMetadataManager`.
* Remove topic hash from cache if related topic image is updated.
* Ignore topic hash 0 when calculating group metadata hash.
* Add `metadataHash` to `ModernGroup`.
* Replace subscription metadata with metadata hash.
* If there is data in `ConsumerGroupPartitionMetadataValue`, set a flag
in group to add tombstone record in next group heartbeat.
Reviewers: David Jacot <djacot@confluent.io>
This patch updates the OffsetFetch API to ensure that a committed offset
is returned iff the requested topic id matches the persisted one; the
invalid offset is returned otherwise.
Reviewers: Lianet Magrans <lmagrans@confluent.io>
The mkMapOfPartitionRacks in ServerSideAssignorBenchmark.java was
introduced in 8013657f5d, and the one in
GroupCoordinatorRecordHelpersTest.java was introduced in
3709901c9e.
Both have not been used since bb97d63d41.
Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
Currently, we were asserting on records containing set using
`assertEquals` which can fail intermittently. To fix the assertion has
been replaced by `assertRecordEquals`.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
https://github.com/apache/kafka/pull/19781/files#diff-00f0f81cf13e66781777d94f7d2e68a581663385c37e98792507f2294c91bb09L2746-R2745
changes the `addInitializingTopicsRecords` signature while
https://github.com/apache/kafka/pull/18929/files#r2104172356 didn't make
adjustment accordingly.
Fix GroupMetadataManager#completeAlterShareGroupOffsets to use
`InitMapValue` in `initializingTopics` so that
`addInitializingTopicsRecords` can accept it.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Sushant Mahajan <sushant.mahajan88@gmail.com>, Kuan-Po Tseng <brandboat@gmail.com>, TengYao Chi <frankvicky@apache.org>, Shivsundar R <shr@confluent.io>, PoAn Yang <payang@apache.org>, Nick Guo <lansg0504@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>
[KAFKA-16720](https://issues.apache.org/jira/browse/KAFKA-16720) aims to
finish the AlterShareGroupOffsets RPC.
Reviewers: Andrew Schofield <aschofield@confluent.io>
---------
Co-authored-by: jimmy <wangzhiwang@qq.com>
1. Currently, the code allows for retrying any initializing topics in
subsequent heartbeats. This can result in duplicate calls to persister
if multiple share consumers join the same group concurrently.
Furthermore, only one of these will succeed as the others will have a
lower state epoch and will be fenced.
2. The existing change was made in
https://github.com/apache/kafka/pull/19603 to allow for retrying
initialization of initializing topics, in case the original caller was
not able to persist the information in the persister due to a dead
broker/timeout.
3. To prevent multiple calls as well as allow for retry we have
supplemented the timelinehashmap holding the
`ShareGroupStatePartitionMetadataInfo` to also hold the timestamp at
which this record gets replayed.
a. Now when we get multiple consumers for the same group and topic,
only one of them is allowed to make the persister initialize request and
this information is added to the map when it is replayed. Thus solving
issue 1.
b. To allow for retries, if an initializing topic is found with a
timestamp which is older than 2*offset_write_commit_ms, that topic will
be allowed to be retried. Here too only one consumer would be able to
retry thus resolving issue 2 as well.
4. Tests have been added wherever applicable and existing ones updated.
5. No record schema changes are involved.
6. The `ShareGroupStatePartitionMetadataInfo` and `InitMapValue` records
have been moved to the `ShareGroup` class for better encapsulation.
7. Some logs have been changed from error to info in
`ShareCoordinatorShard` and extra information is logged.
Reviewers: Andrew Schofield <aschofield@confluent.io>
Replace `TopicMetadata` with `MetadataImage` in
`SubscribedTopicDescriberImpl` and `TargetAssignmentBuilder`.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot
<djacot@confluent.io>
Enable next system test with KIP-1071.
Some of the validation inside the test did not make sense for KIP-1071.
This is because in KIP-1071, if a member leaves or joins the group, not
all members may enter a REBALANCING state. We use the wrapper introduced
in [KAFKA-19271](https://issues.apache.org/jira/browse/KAFKA-19271)
to print a log line whenever the member epoch is bumped, which is the
only way a member can "indirectly" observe that other members are
rebalancing.
Reviewers: Bill Bejeck <bill@confluent.io>
This PR adds changes, so the IQ endpoint information is only sent to
streams group members when there has been a change in the assignments
requiring an update in the streams client host-partition ownership.
The existing IQ integration test passes with no modifications and
updated the `GroupMetadataManagerTest` to cover the new process.
Reviewers: Matthias Sax <mjsax@apache.org>, Lucas Brutschy
<lucasbru@apache.org>
This patch adds the `TopicId` field to the `OffsetCommitValue` record as
a tagged field. It will be later used on the offset fetch path to ensure
that the persisted offset matches the requested one.
Reviewers: Dongnuo Lyu <dlyu@confluent.io>, Sean Quah
<squah@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
* Change `ListClientMetricsResourcesRequest.json` to
`ListConfigResourcesRequest.json`.
* Change `ListClientMetricsResourcesResponse.json` to
`ListConfigResourcesResponse.json`.
* Change `ListClientMetricsResourcesRequest.java` to
`ListConfigResourcesRequest.java`.
* Change `ListClientMetricsResourcesResponse.java` to
`ListConfigResourcesResponsejava`.
* Change `KafkaApis` to handle both `ListClientMetricsResourcesRequest`
v0 and v1 requests.
Reviewers: Andrew Schofield <aschofield@confluent.io>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
* Previously we had added code to `GroupCoordinatorService.onElection`
to reconcile pending share group initializing topics. This was done to
manage state in case of failovers and broker failures.
* However, we later modified share group heartbeat code to do the retry
to clean up the state and the `onElection` code is now redundant.
* In this PR we are cleaning up this code.
Reviewers: David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>
* Add `com.dynatrace.hash4j:hash4j:0.22.0` to dependencies.
* Add `computeTopicHash` to `org.apache.kafka.coordinator.group.Utils`.
* If topic name is non-existent, return 0.
* If topic name is existent, use streaming XXH3 to compute topic hash
with magic byte, topic id, topic name, number of partitions, partition
id and sorted racks.
* Add `computeGroupHash` to `org.apache.kafka.coordinator.group.Utils`.
* If topic map is empty, return 0.
* If topic map is not empty, use streaming XXH3 to compute group
metadata hash with sorted topic hashes by topic names.
* Add related unit test.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>, Sean Quah <squah@confluent.io>, David Jacot <djacot@confluent.io>
---------
Signed-off-by: PoAn Yang <payang@apache.org>
When the number of partitions is not divisible by the number of members,
some members will end up with one more partition than others.
Previously, we required these to be the members at the start of the
iteration order, which meant that partitions could be reassigned even
when the previous assignment was already balanced.
Allow any member to have the extra partition, so that we do not move
partitions around when the previous assignment is already balanced.
Before the PR
```
Benchmark (assignmentType) (assignorType) (isRackAware) (memberCount) (partitionsToMemberRatio) (subscriptionType) (topicCount) Mode Cnt Score Error Units
ServerSideAssignorBenchmark.doAssignment FULL RANGE false 10000 50 HOMOGENEOUS 1000 avgt 2 26.175 ms/op
ServerSideAssignorBenchmark.doAssignment FULL RANGE false 10000 50 HETEROGENEOUS 1000 avgt 2 123.955 ms/op
ServerSideAssignorBenchmark.doAssignment INCREMENTAL RANGE false 10000 50 HOMOGENEOUS 1000 avgt 2 24.408 ms/op
ServerSideAssignorBenchmark.doAssignment INCREMENTAL RANGE false 10000 50 HETEROGENEOUS 1000 avgt 2 114.873 ms/op
```
After the PR
```
Benchmark (assignmentType) (assignorType) (isRackAware) (memberCount) (partitionsToMemberRatio) (subscriptionType) (topicCount) Mode Cnt Score Error Units
ServerSideAssignorBenchmark.doAssignment FULL RANGE false 10000 50 HOMOGENEOUS 1000 avgt 2 24.259 ms/op
ServerSideAssignorBenchmark.doAssignment FULL RANGE false 10000 50 HETEROGENEOUS 1000 avgt 2 118.513 ms/op
ServerSideAssignorBenchmark.doAssignment INCREMENTAL RANGE false 10000 50 HOMOGENEOUS 1000 avgt 2 24.636 ms/op
ServerSideAssignorBenchmark.doAssignment INCREMENTAL RANGE false 10000 50 HETEROGENEOUS 1000 avgt 2 115.503 ms/op
```
Reviewers: David Jacot <djacot@confluent.io>
* Currently even if a user topic is deleted, its related records are not
deleted with respect to subscribed share groups from the GC and the SC.
* The event of topic delete is propagated from the
BrokerMetadataPublisher to the coordinators via the
`onPartitionsDeleted` method. This PR leverages this method to issue
cleanup calls to the GC and SC respectively.
* To prevent chaining of futures in the GC, we issue async calls to both
GC and SC independently and the methods take care of the respective
cleanups unaware of the other.
* This method is more efficient and transcends issues related to
timeouts/broker restarts resulting in chained future execution issues.
Reviewers: Andrew Schofield <aschofield@confluent.io>
This patch extends the OffsetFetch API to support topic ids. From
version 10 of the API, topic ids must be used.
The patch only contains the server side changes and it keeps the version
10 as unstable for now. We will mark the version as stable when the
client side changes are merged in.
Reviewers: TengYao Chi <frankvicky@apache.org>, Lianet Magrans <lmagrans@confluent.io>
Updated default locks config for share groups from 200 to 2000. The
increase in the limit is a result from tests which showed that with
default maxFetchRecords of 500 from client and 200 as locks limit, there
can't be parallel fetch for same partition. Also the tests resulted that
sharing a partition to an index of 3-4 is easily achievable, hence
raised the limit to 4 times of default limit of maxFetchRecords (500).
Reviewers: Andrew Schofield <aschofield@confluent.io>
When a group has pending transactional offsets but no committed offsets,
we can accidentally delete it while cleaning up expired offsets. Add a
check to avoid this case.
Reviewers: David Jacot <djacot@confluent.io>
When fetching stable offsets in the group coordinator, we iterate over
all requested partitions. For each partition, we iterate over the
group's ongoing transactions to check if there is a pending
transactional offset commit for that partition.
This can get slow when there are a large number of partitions and a
large number of pending transactions. Instead, maintain a list of
pending transactions per partition to speed up lookups.
Reviewers: Shaan, Dongnuo Lyu <dlyu@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, David Jaco <djacot@confluent.io>
When the persister is responding to a read share-group state summary
request, it has no way of including the leader epoch in its response,
even though it has the information to hand. This means that the leader
epoch information is not initialised in the admin client operation to
list share group offsets, and this then means that the information
cannot be displayed in kafka-share-groups.sh.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan
<smahajan@confluent.io>
* Currently in the share group heartbeat flow, if we see a TP subscribed
for the first time, we move that TP to initializing state in GC and let
the GC send a persister request to share group initialize the
aforementioned TP.
* However, if the coordinator runtime request for share group heartbeat
times out (maybe due to restarting/bad broker), the future completes
exceptionally resulting in persiter request to not be sent.
* Now, we are in a bad state since the TP is in initializing state in GC
but not persister initialized. Future heartbeats for the same share
partitions will also not help since we do not allow retrying persister
request for initializing TPs.
* This PR remedies the situation by allowing the same.
* A temporary fix to increase offset commit timeouts in system tests was
added to fix the issue. In this PR, we revert that change as well.
Reviewers: Andrew Schofield <aschofield@confluent.io>