Commit Graph

3792 Commits

Author SHA1 Message Date
David Jacot 98e535b524
MINOR: Simplify OffsetFetchResponse (#19642)
While working on https://github.com/apache/kafka/pull/19515, I came to
the conclusion that the OffsetFetchResponse is quite messy and overall
too complicated. This patch rationalize the constructors.
OffsetFetchResponse has a single constructor accepting the
OffsetFetchResponseData. A builder is introduced to handle the down
conversion. This will also simplify adding the topic ids. All the
changes are mechanical, replacing data structures by others.

Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-05-08 14:57:45 +02:00
Apoorv Mittal 2dd6126b5d
KAFKA-18855 Slice API for MemoryRecords (#19581)
CI / build (push) Waiting to run Details
The PR adds `slice` API in `Records.java` and further implementation in
`MemoryRecords`. With the addition of ShareFetch and it's support to
read from TieredStorage, where ShareFetch might acquire subset of fetch
batches and TieredStorage emits MemoryRecords, hence a slice API is
needed for MemoryRecords as well to limit the bytes transferred (if
subset batches are acquired).

MemoryRecords are sliced using `duplicate` and `slice` API of
ByteBuffer, which are backed by the original buffer itself hence no-copy
is created rather position, limit and offset are changed as per the new
position and length.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao
 <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-08 14:02:25 +08:00
Lianet Magrans 67b46fec15
MINOR: introduce structure to keep member assignment with topic Ids (#19645)
- Add new DS to wrap the member assignment (containing topic Ids, names
and partitions), to easily access the data as needed. This will be used
in following PR to integrate assignment with topic IDs into the
subscription state.
- Improve logging on the client assignment/reconciliation path

No changes in logic.

Reviewers: TengYao Chi <frankvicky@apache.org>, Andrew Schofield
 <aschofield@confluent.io>
2025-05-07 13:57:56 -04:00
Kirk True d3707fc815
KAFKA-19214: Clean up use of Optionals in RequestManagers.entries() (#19609)
Change:

`public List<Optional<? extends RequestManager>> entries();`

to:

`public List<RequestManager> entries();`

and clean up the callers.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Andrew Schofield
 <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-07 17:18:12 +01:00
yunchi d034268312
MINOR: Remove ConstantBrokerOrActiveKController (#19654)
`ConstantBrokerOrActiveKController` was introduced in #14399, to provide
a mechanism for selecting the least loaded broker or the active
controller when using `bootstrap.controllers`.

Usage was removed in #18002, after `alterConfigs` was deprecated in
Kafka 2.4.0.

Reviewers: PoAn Yang <payang@apache.org>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>, Ken Huang <s7133700@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-07 20:23:29 +08:00
Lan Ding e1da318722
MINOR: add boundary IT for delivery count (#19649)
CI / build (push) Waiting to run Details
see
https://github.com/apache/kafka/pull/19430#pullrequestreview-2809619176
Add boundary IT for delivery count.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-06 22:05:02 +01:00
Andrew Schofield 7d027a4d83
KAFKA-19218: Add missing leader epoch to share group state summary response (#19602)
CI / build (push) Waiting to run Details
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>
2025-05-06 14:53:12 +01:00
Dmitry Werner 0810650da1
MINOR: Small cleanups in clients tests (#19634)
- Removed unused fields and methods in clients tests
- Fixed IDEA code inspection warnings

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, Andrew Schofield <aschofield@confluent.io>,
 Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
 <frankvicky@apache.org>
2025-05-06 20:19:21 +08:00
yunchi 4e77466f6a
KAFKA-19170 Move MetricsDuringTopicCreationDeletionTest to client-integration-tests module (#19528)
rewrite `MetricsDuringTopicCreationDeletionTest` to `ClusterTest` infra
and move it to clients-integration-tests module.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
<s7133700@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping
Tsai <chia7712@gmail.com>
2025-05-06 19:57:16 +08:00
Alieh Saeedi 54b3b3debc
MINOR: Convert streams group options to consumer group options in Admin APIs (#19583)
This PR is fixing the issue introduced in #19120
The input `StreamsGroup`-options must not be ignored, but it must be
converted to `ConsumerGroup`-options.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-05-06 13:26:56 +02:00
Andrew Schofield d2bd68d50c
MINOR: Improve output for delete-offset of kafka-consumer-groups.sh (#19610)
The output from the delete-offsets option of kafka-consumer-groups.sh
can be improved. For example, the column widths are excessive which
looks untidy, and the output messages can be improved.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-06 12:20:36 +01:00
TaiJuWu 19530738c4
KAFKA-19240 Move MetadataVersionIntegrationTest to clients-integration-tests module (#19641)
The PR do following:
1. Move MetadataVersionIntegrationTest to clients-integration-tests
module
2. rewrite to java from scala

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-06 00:12:57 +08:00
Shivsundar R fedbb90c12
KAFKA-19232: Handle Share session limit reached exception in clients. (#19619)
CI / build (push) Waiting to run Details
Handle the new `ShareSessionLimitReachedException` in
`ShareSessionHandler` in the client to reset the ShareSession.  Added a
unit test verifying the change.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-04 19:59:40 +01:00
yunchi bff5ba4ad9
MINOR: replace .stream().forEach() with .forEach() (#19626)
CI / build (push) Waiting to run Details
replace all applicable `.stream().forEach()` in codebase with just
`.forEach()`.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-04 20:39:55 +08:00
Ken Huang c85e09f7a5
KAFKA-19060 Documented null edge cases in the Clients API JavaDoc (#19393)
Some client APIs may return `null` values in the map, but this  behavior
isn’t documented in the JavaDoc. We should update the JavaDoc to include
these edge cases.

Reviewers: Kirk True <kirk@kirktrue.pro>, Jhen-Yung Hsu
<jhenyunghsu@gmail.com>, PoAn Yang <payang@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-05-04 20:35:02 +08:00
xijiu b5cceb43e5
KAFKA-19205: inconsistent result of beginningOffsets/endoffset between classic and async consumer with 0 timeout (#19578)
CI / build (push) Waiting to run Details
In the return results of the methods beginningOffsets and endOffset, if
timeout == 0, then an empty Map should be returned uniformly instead of
in the form of <TopicPartition, null>

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>, Lianet
 Magrans <lmagrans@confluent.io>
2025-05-03 13:12:20 -04:00
TengYao Chi 93e65c4539
KAFKA-18267 Add unit tests for CloseOptions (#19571)
There is some redundant code that could be removed in `CloseOptions`.
This patch also adds unit tests for CloseOptions.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-03 22:36:43 +08:00
Matthias J. Sax 44025d8116
MINOR: fix bug in MockConsumer (#19627)
The setter of `maxPollRecords` wrongly checks the field instead of the argument.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
 <frankvicky@apache.org>
2025-05-03 14:08:18 +08:00
Sushant Mahajan e68781414e
KAFKA-19204: Allow persister retry of initializing topics. (#19603)
CI / build (push) Waiting to run Details
* 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>
2025-05-02 14:25:29 +01:00
Matthias J. Sax f69337b37c
MINOR: use `isEmpty()` to avoid compiler warning (#19616)
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2025-05-01 23:51:36 -07:00
Calvin Liu 0c1fbf3aeb
KAFKA-19073 add transactional ID pattern filter to ListTransactions (#19355)
Propose adding a new filter TransactionalIdPattern. This transaction ID pattern filter works as AND with the other transaction filters. Also, it is empowered with Re2j.

KIP: https://cwiki.apache.org/confluence/x/4gm9F

Reviewers: Justine Olshan <jolshan@confluent.io>, Ken Huang
<s7133700@gmail.com>, Kuan-Po Tseng <brandboat@gmail.com>, Chia-Ping
Tsai <chia7712@gmail.com>
2025-05-02 00:52:21 +08:00
Lan Ding 8dbf56e4b5
KAFKA-17541:[1/2] Improve handling of delivery count (#19430)
For records which are automatically released as a result of closing a
share session normally, the delivery count should not be incremented.
These records were fetched but they were not actually delivered to the
client since the disposition of the delivery records is carried in the
ShareAcknowledge which closes the share session. Any remaining records
were not delivered, only fetched.
This PR releases the delivery count for records when closing a share
session normally.

Co-authored-by: d00791190 <dinglan6@huawei.com>

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-05-01 14:40:03 +01:00
Chirag Wadhwa 800612e4a7
KAFKA-19015: Remove share session from cache on share consumer connection drop (#19329)
Up till now, the share sessions in the broker were only attempted to
evict when the share session cache was full and a new session was trying
to get registered. With the changes in this PR, whenever a share
consumer gets disconnected from the broker, the corresponding share
session would be evicted from the cache.

Note - `connectAndReceiveWithoutClosingSocket` has been introduced in
`GroupCoordinatorBaseRequestTest`. This method creates a socket
connection, sends the request, receives a response but does not close
the connection. Instead, these sockets are stored in a ListBuffer
`openSockets`, which are closed in tearDown method after each test is
run. Also, all the `connectAndReceive` calls in
`ShareFetchAcknowledgeRequestTest` have been replaced by
`connectAndReceiveWithoutClosingSocket`, because these tests depends
upon the persistence of the share sessions on the broker once
registered. But, with the new code introduced, as soon as the socket
connection is closed, a connection drop is assumed by the broker,
leading to session eviction.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-05-01 14:36:18 +01:00
Lianet Magrans 1059af4eac
MINOR: Improve docs for client group configs (#19605)
CI / build (push) Waiting to run Details
Improve java docs for session and HB interval client configs & fix
max.poll.interval description

Reviewers: David Jacot <djacot@confluent.io>
2025-04-30 14:04:16 -04:00
Andrew Schofield ce97b1d5e7
KAFKA-16894: Exploit share feature [3/N] (#19542)
This PR uses the v1 of the ShareVersion feature to enable share groups
for KIP-932.

Previously, there were two potential configs which could be used -
`group.share.enable=true` and including "share" in
`group.coordinator.rebalance.protocols`. After this PR, the first of
these is retained, but the second is not. Instead, the preferred switch
is the ShareVersion feature.

The `group.share.enable` config is temporarily retained for testing and
situations in which it is inconvenient to set the feature, but it should
really not be necessary, especially when we get to AK 4.2. The aim is to
remove this internal config at that point.

No tests should be setting `group.share.enable` any more, because they
can use the feature (which is enabled in test environments by default
because that's how features work). For tests which need to disable share
groups, they now set the share feature to v0. The majority of the code
changes were related to correct initialisation of the metadata cache in
tests now that a feature is used.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-04-30 13:27:01 +01:00
PoAn Yang 81881dee83
KAFKA-18760: Deprecate Optional<String> and return String from public Endpoint#listener (#19191)
* Deprecate org.apache.kafka.common.Endpoint#listenerName.
* Add org.apache.kafka.common.Endpoint#listener to replace
org.apache.kafka.common.Endpoint#listenerName.
* Replace org.apache.kafka.network.EndPoint with
org.apache.kafka.common.Endpoint.
* Deprecate org.apache.kafka.clients.admin.RaftVoterEndpoint#name
* Add org.apache.kafka.clients.admin.RaftVoterEndpoint#listener to
replace org.apache.kafka.clients.admin.RaftVoterEndpoint#name

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TaiJuWu
 <tjwu1217@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao
 Chi <frankvicky@apache.org>, Ken Huang <s7133700@gmail.com>, Bagda
 Parth  , Kuan-Po Tseng <brandboat@gmail.com>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-04-30 12:15:33 +08:00
Ken Huang 676e0f2ad6
KAFKA-19139 Plugin#wrapInstance should use LinkedHashMap instead of Map (#19519)
CI / build (push) Waiting to run Details
There will be an update to the PluginMetrics#metricName method: the type
of the tags parameter will be changed
from Map to LinkedHashMap.
This change is necessary because the order of metric tags is important
1. If the tag order is inconsistent, identical metrics may be treated as
distinct ones by the metrics backend
2. KAFKA-18390 is updating metric naming to use LinkedHashMap. For
consistency, we should follow the same approach here.

Reviewers: TengYao Chi <frankvicky@apache.org>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>, lllilllilllilili
2025-04-30 10:43:01 +08:00
Bill Bejeck 431cffc93f
KAFKA-19135 Migrate initial IQ support for KIP-1071 from feature branch to trunk (#19588)
This PR is a migration of the initial IQ support for KIP-1071 from the
feature branch to trunk.  It includes a parameterized integration test
that expects the same results whether using either the classic or new
streams group protocol.

Note that this PR will deliver IQ information in each heartbeat
response.  A follow-up PR will change that to be only sending IQ
information when assignments change.

Reviewers Lucas Brutschy <lucasbru@apache.org>
2025-04-29 20:08:49 -04:00
Matthias J. Sax 3bb15c5dee
MINOR: improve JavaDocs for consumer CloseOptions (#19546)
Reviewers: TengYao Chi <frankvicky@apache.org>, PoAn Yang <payang@apache.org>, Lianet Magrans <lmagrans@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2025-04-29 16:38:16 -07:00
Omnia Ibrahim 6f783f8536
KAFKA-10551: Add topic id support to produce request and response (#15968)
- Add support topicId in `ProduceRequest`/`ProduceResponse`. Topic name
and Topic Id will become `ignorable` following the footstep of
`FetchRequest`/`FetchResponse`
- ReplicaManager still look for `HostedPartition` using `TopicPartition`
and doesn't check topic id. This is an **[OPEN QUESTION]** if we should
address this in this pr or wait for
[KAFKA-16212](https://issues.apache.org/jira/browse/KAFKA-16212) as this
will update `ReplicaManager::getPartition` to use `TopicIdParittion`
once we update the cache. Other option is that we compare provided
`topicId` with `Partition` topic id and return `UNKNOW_TOPIC_ID` or
`UNKNOW_TOPIC_PARTITION` if we can't find partition with matched topic
id.

Reviewers: Jun Rao <jun@confluent.io>, Justine Olshan
 <jolshan@confluent.io>
2025-04-29 15:37:28 -07:00
Ritika Reddy 2fdb687029
KAFKA-19082: [2/4] Add preparedTxnState class to Kafka Producer (KIP-939) (#19470)
CI / build (push) Waiting to run Details
This is part of the client side changes required to enable 2PC for
KIP-939

New KafkaProducer.PreparedTxnState class is going to be defined as
following:  ```  static public class PreparedTxnState {    public String
toString();    public PreparedTxnState(String serializedState); public
PreparedTxnState();  }  ```  The objects of this class can serialize to
/ deserialize from a string value and can be written to / read from a
database.  The implementation is going to store producerId and epoch in
the format **producerId:epoch**

Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan
 <jolshan@confluent.io>
2025-04-29 11:52:02 -07:00
David Jacot 6d67d82d5b
MINOR: Cleanup OffsetFetchRequest/Response in MessageTest (#19576)
CI / build (push) Waiting to run Details
The tests related of OffsetFetch request/response in MessageTest are
incomprehensible. This patch rewrites them in a simpler way.

Reviewers: TengYao Chi <frankvicky@apache.org>
2025-04-28 13:14:23 +02:00
David Jacot be194f5dba
MINOR: Simplify OffsetFetchRequest (#19572)
While working on https://github.com/apache/kafka/pull/19515, I came to
the conclusion that the OffsetFetchRequest is quite messy and overall
too complicated. This patch rationalize the constructors.
OffsetFetchRequest has a single constructor accepting the
OffsetFetchRequestData. This will also simplify adding the topic ids.
All the changes are mechanical, replacing data structures by others.

Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi <frankvicky@apache.org>, Lianet Magran <lmagrans@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-04-27 18:58:30 +02:00
Chirag Wadhwa 2f9c2dd828
KAFKA-16718-3/n: Added the ShareGroupStatePartitionMetadata record during deletion of share group offsets (#19478)
This is a follow up PR for implementation of DeleteShareGroupOffsets
RPC. This PR adds the ShareGroupStatePartitionMetadata record to
__consumer__offsets topic to make sure the topic is removed from the
initializedTopics list. This PR also removes partitions from the request
and response schemas for DeleteShareGroupState RPC

Reviewers: Sushant Mahajan <smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-04-25 22:01:48 +01:00
Ken Huang b4b80731c1
KAFKA-19042 Move PlaintextConsumerFetchTest to client-integration-tests module (#19520)
Use Java to rewrite `PlaintextConsumerFetchTest` by new test infra and
move it to client-integration-tests module.

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-04-26 00:09:23 +08:00
Lucas Brutschy 732ed0696b
KAFKA-19190: Handle shutdown application correctly (#19544)
If the streams rebalance protocol is enabled in
StreamsUncaughtExceptionHandlerIntegrationTest, the streams application
does not shut down correctly upon error.

There are two causes for this. First, sometimes, the SHUTDOWN_APPLICATION
code only sent with the leave heartbeat, but that is not handled broker
side. Second, the SHUTDOWN_APPLICATION code wasn't properly handled
client-side at all.

Reviewers: Bruno Cadonna <cadonna@apache.org>, Bill Bejeck
 <bill@confluent.io>, PoAn Yang <payang@apache.org>
2025-04-25 09:56:09 +02:00
PoAn Yang 36d2498fb3
MINOR: Use meaningful name in AsyncKafkaConsumerTest (#19550)
Replace names like a, b, c, ... with meaningful names in
AsyncKafkaConsumerTest.

Follow-up:
https://github.com/apache/kafka/pull/19457#discussion_r2056254087

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

Reviewers: Bill Bejeck <bbejeck@apache.org>, Ken Huang <s7133700@gmail.com>
2025-04-24 17:17:33 -04:00
David Jacot a948537704
MINOR: Small refactor in group coordinator (#19551)
This patch does a few code changes:
* It cleans up the GroupCoordinatorService;
* It moves the helper methods to validate request to Utils;
* It moves the helper methods to create the assignment for the
ConsumerGroupHeartbeatResponse and the ShareGroupHeartbeatResponse from
the GroupMetadataManager to the respective classes.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jeff Kim <jeff.kim@confluent.io>
2025-04-24 20:57:23 +02:00
Ritika Reddy 62fe528f4b
KAFKA-19082: [1/4] Add client config for enable2PC and overloaded initProducerId (KIP-939) (#19429)
This is part of the client side changes required to enable 2PC for
KIP-939

**Producer Config:**
transaction.two.phase.commit.enable The default would be ‘false’.  If
set to ‘true’, the broker is informed that the client is participating
in two phase commit protocol and transactions that this client starts
never expire.

**Overloaded InitProducerId method**
If the value is 'true' then the corresponding field is set in the
InitProducerIdRequest

Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
 <alivshits@confluent.io>
2025-04-24 09:41:06 -07:00
Apoorv Mittal 3c05dfdf0e
KAFKA-18889: Make records in ShareFetchResponse non-nullable (#19536)
This PR marks the records as non-nullable for ShareFetch.

This PR is as per the changes for Fetch:
https://github.com/apache/kafka/pull/18726 and some work for ShareFetch
was done here: https://github.com/apache/kafka/pull/19167. I tested with
marking `records` as non-nullable in ShareFetch, which required
additional handling. The same has been fixed in current PR.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai
 <chia7712@gmail.com>, TengYao Chi <frankvicky@apache.org>, PoAn Yang
 <payang@apache.org>
2025-04-24 16:32:08 +01:00
Vikas Singh f4ab3a2275
MINOR: Use readable interface to parse response (#19353)
The generated response data classes take Readable as input to parse the
Response. However, the associated response objects take ByteBuffer as
input and thus convert them to Readable using `new ByteBufferAccessor`
call.

This PR changes the parse method of all the response classes to take the
Readable interface instead so that no such conversion is needed.

To support parsing the ApiVersionsResponse twice for different version
this change adds the "slice" method to the Readable interface.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Truc Nguyen
<[trnguyen@confluent.io](mailto:trnguyen@confluent.io)>, Aadithya
Chandra <[aadithya.c@gmail.com](mailto:aadithya.c@gmail.com)>
2025-04-24 11:09:06 -04:00
Andrew Schofield f0f5571dbb
MINOR: Change KIP-932 log messages from early access to preview (#19547)
Change the log messages which used to warn that KIP-932 was an Early
Access feature to say that it is now a Preview feature. This will make
the broker logs far less noisy when share groups are enabled.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-04-24 11:22:17 +01:00
PoAn Yang 3fae785ea0
KAFKA-19110: Add missing unit test for Streams-consumer integration (#19457)
- Construct `AsyncKafkaConsumer` constructor and verify that the
`RequestManagers.supplier()` contains Streams-specific data structures.
- Verify that `RequestManagers` constructs the Streams request managers
correctly
- Test `StreamsGroupHeartbeatManager#resetPollTimer()`
- Test `StreamsOnTasksRevokedCallbackCompletedEvent`,
`StreamsOnTasksAssignedCallbackCompletedEvent`, and
`StreamsOnAllTasksLostCallbackCompletedEvent` in
`ApplicationEventProcessor`
- Test `DefaultStreamsRebalanceListener`
- Test `StreamThread`.
  - Test `handleStreamsRebalanceData`.
  - Test `StreamsRebalanceData`.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Bill Bejeck <bill@confluent.io>
Signed-off-by: PoAn Yang <payang@apache.org>
2025-04-24 10:38:22 +02:00
Kirk True 8b4560e3f0
KAFKA-15767 Refactor TransactionManager to avoid use of ThreadLocal (#19440)
Introduces a concrete subclass of `KafkaThread` named `SenderThread`.
The poisoning of the TransactionManager on invalid state changes is
determined by looking at the type of the current thread.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-04-24 00:31:30 +08:00
Bruno Cadonna efd785274e
KAFKA-19124: Follow up on code improvements (#19453)
Improves a variable name and handling of an Optional.

Reviewers: Bill Bejeck <bill@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang <payang@apache.org>
2025-04-23 14:24:33 +02:00
David Jacot 71d08780d1
KAFKA-14690; Add TopicId to OffsetCommit API (#19461)
This patch extends the OffsetCommit API to support topic ids. From
version 10 of the API, topic ids must be used. Originally, we wanted to
support both using topic ids and topic names from version 10 but it
turns out that it makes everything more complicated. Hence we propose to
only support topic ids from version 10. Clients which only support using
topic names can either lookup the topic ids using the Metadata API or
stay on using an earlier version.

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: Lianet Magrans <lmagrans@confluent.io>, PoAn Yang <payang@apache.org>
2025-04-23 08:22:09 +02:00
Andrew Schofield e78e106221
MINOR: Improve javadoc for share consumer (#19533)
Small improvements to share consumer javadoc.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-04-22 15:54:05 +01:00
Andrew Schofield 66147d5de7
KAFKA-19057: Stabilize KIP-932 RPCs for AK 4.1 (#19378)
This PR removes the unstable API flag for the KIP-932 RPCs.

The 4 RPCs which were exposed for the early access release in AK 4.0 are
stabilised at v1. This is because the RPCs have evolved over time and AK
4.0 clients are not compatible with AK 4.1 brokers. By stabilising at
v1, the API version checks prevent incompatible communication and
server-side exceptions when trying to parse the requests from the older
clients.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-04-22 11:43:32 +01:00
Rich Chen ae771d73d1
KAFKA-8830 make Record Headers available in onAcknowledgement (#17099)
Two sets of tests are added:
1. KafkaProducerTest
- when send success, both record.headers() and onAcknowledgement headers
are read only
- when send failure, record.headers() is writable as before and
onAcknowledgement headers is read only
2. ProducerInterceptorsTest
- make both old and new onAcknowledgement method are called successfully

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Omnia Ibrahim
<o.g.h.ibrahim@gmail.com>, Matthias J. Sax <matthias@confluent.io>,
Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-04-21 21:01:55 +08:00
Hong-Yi Chen 8fa0d9723f
MINOR: Fix typo in ApiKeyVersionsProvider exception message (#19521)
This patch addresses issue #19516 and corrects a typo in
`ApiKeyVersionsProvider`: when `toVersion` exceeds  `latestVersion`, the
`IllegalArgumentException` message was erroneously formatted with
`fromVersion`. The format argument has been updated to use `toVersion`
so that the error message reports the correct value.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Chia-Ping
 Tsai <chia7712@gmail.com>
2025-04-21 15:35:47 +08:00