Commit Graph

3875 Commits

Author SHA1 Message Date
Luke Chen cdc7a4e2b7 MINOR: improve the min.insync.replicas doc (#20237)
Along with the change: https://github.com/apache/kafka/pull/17952

([KIP-966](https://cwiki.apache.org/confluence/display/KAFKA/KIP-966%3A+Eligible+Leader+Replicas)),
the semantics of `min.insync.replicas` config has small change, and add
some constraints. We should document them clearly.

Reviewers: Jun Rao <junrao@gmail.com>, Calvin Liu <caliu@confluent.io>,
 Mickael Maison <mickael.maison@gmail.com>, Paolo Patierno
 <ppatierno@live.com>, Federico Valeri <fedevaleri@gmail.com>, Chia-Ping
 Tsai <chia7712@gmail.com>
2025-08-05 00:27:04 +08:00
Bill Bejeck f35f94b3e6 KAFKA-19504: Remove unused metrics reporter initialization in KafkaAdminClient (#20166)
CI / build (push) Has been cancelled Details
The `AdminClient` adds a telemetry reporter to the metrics reporters
list in the constructor.  The problem is that the reporter was already
added in the `createInternal` method.  In the `createInternal` method
call, the `clientTelemetryReporter` is added to a
`List<MetricReporters>` which is passed to the `Metrics` object, will
get closed when `Metrics.close()` is called.  But adding a reporter to
the reporters list in the constructor is not used by the `Metrics`
object and hence doesn't get closed, causing a memory leak.

All related tests pass after this change.

Reviewers: Apoorv Mittal <apoorvmittal10@apache.org>, Matthias J. Sax
 <matthias@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>,
 Jhen-Yung Hsu <jhenyunghsu@gmail.com>
2025-07-14 20:21:12 -04:00
Ismael Juma 487af011ca KAFKA-19444: Add back JoinGroup v0 & v1 (#20116)
CI / build (push) Has been cancelled Details
This fixes librdkafka older than the recently released 2.11.0 with
Kerberos authentication and Apache Kafka 4.x.

Even though this is a bug in librdkafka, a key goal of KIP-896 is not to
break the popular client libraries listed in it. Adding back JoinGroup
v0 & v1 is a very small change and worth it from that perspective.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-07-07 15:15:29 -07:00
Omnia Ibrahim e6b78ae9e5 KAFKA-19397: Ensure consistent metadata usage in produce request and response (#19964)
CI / build (push) Waiting to run Details
- Metadata doesn't have the full view of topicNames to ids during
rebootstrap of client or when topic has been deleted/recreated. The
solution is to pass down topic id and stop trying to figure it out later
in the logic.

---------

Co-authored-by: Kirk True <kirk@kirktrue.pro>
2025-07-07 19:52:15 +08:00
Ken Huang f14e60fc8f KAFKA-19042 Move ProducerSendWhileDeletionTest to client-integration-tests module (#19971)
Use Java to rewrite ProducerSendWhileDeletionTest by new test infra and
move it to client-integration-tests module.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-07-07 16:26:51 +08:00
David Jacot 7bdeb36a52 KAFKA-19246; OffsetFetch API does not return group level errors correctly with version 1 (#19704)
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>
2025-06-26 15:30:44 +02:00
Ritika Reddy c4cac07819
KAFKA-19414: Remove 2PC public APIs from 4.1 until release (KIP-939) (#19985)
We are removing some of the previously added public APIs until KIP-939
is ready to use.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-06-25 09:06:21 -07:00
Lianet Magrans 38652f763d KAFKA-19356: Prevent new consumer fetch assigned partitions not in explicit subscription (#19983)
Fix to ensure assigned partitions whose topics are not in the consumer
explicit subscription are considered not fetchable (so that no records
are returned on poll for them)

This scenario could happen in the new async consumer (using the Consumer
rebalance protocol) when the subscription changes, because the consumer
will keep its assignment until the coordinator sends a new one (broker
drives assignments).

This does not happen in the classic consumer because the assignment
logic lives on the client-side, so the consumer pro-actively updates
assignment as needed.

This PR validates assignment vs subscription on fetch for explicit
subscription  only. Regular expressions, shared subscription  remain
unchanged (regex case still under discussion, will be handled separately
if needed)

Reviewers: Andrew Schofield <aschofield@confluent.io>, TengYao Chi
 <frankvicky@apache.org>, Kirk True <ktrue@confluent.io>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>
2025-06-18 18:39:06 -04:00
Kirk True 198389663e KAFKA-19297: Refactor AsyncKafkaConsumer's use of Java Streams APIs in critical sections (#19917)
Profiling has shown that using the Collections Streams API approach adds
unnecessary overhead compared to a traditional for loop. Minor revisions
to the code have been made to use simpler constructs to improve
performance.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>
2025-06-18 10:01:33 -04:00
Lucas Brutschy 8c4224057d MINOR: Add entity types in streams group RPCs (#19979)
These can be used to implement transformations on top of the RPC
definitions. Group IDs were already marked. This PR additionally adds
the entityType for all topic names.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2025-06-18 09:16:02 +02:00
José Armando García Sancio 88eced0c0f KAFKA-14145; Faster KRaft HWM replication (#19800)
This change compares the remote replica's HWM with the leader's HWM and
completes the FETCH request if the remote HWM is less than the leader's
HWM. When the leader's HWM is updated any pending FETCH RPC is
completed.

Reviewers: Alyssa Huang <ahuang@confluent.io>, David Arthur
 <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>
(cherry picked from commit 742b327025)
2025-06-17 13:27:11 -04:00
Dave Troiano 4308dc39a7 MINOR: in min.insync.replicas config doc, explicitly state that all ISR must ack when acks=all (#19973)
Clarify the interaction of `min.insync.replicas` and `ack=all`
configuration.  Prior to this change, the doc for `min.insync.replicas`
could have been  interpreted as being used to short-circuit in the
`acks=all` case as if  it would be enough if `min.inscyn.replicas`
number of brokers replicated  a message before it can be acknowledged
back to the producer.

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

---------

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
2025-06-16 18:43:33 -07:00
Kirk True 923b6c3fea KAFKA-19153: Add OAuth integration tests (#19938)
Adds a test dependency on
[mock-oauth2-server](https://github.com/navikt/mock-oauth2-server/) for
integration tests for OAuth layer. Also includes fixes for some
regressions that were caught by the integration tests.

Reviewers: Manikumar Reddy <manikumar@confluent.io>, Lianet Magrans
 <lmagrans@confluent.io>
2025-06-12 15:50:07 -04:00
Lianet Magrans 1efbbb8ec1 KAFKA-18117; KAFKA-18729: Use assigned topic IDs to avoid full metadata requests on broker-side regex (#19814)
This PR uses topic IDs received in assignment (under new protocol) to
ensure that only these assigned topics are included in the consumer
metadata requests performed when the user subscribes to broker-side
regex (RE2J).

For handling the edge case of consumer needing metadata for topics IDs
(from RE2J) and topic names (from transient topics), the approach is to
send a request for the transient topics needed temporarily, and once
those resolved, the request for the topic IDs needed for RE2J will
follow. (this is because the broker doesn't accept requests for names
and IDs at the same time)

With the changes we also end up fixing another issue (KAFKA-18729) aimed
at avoiding iterating the full set of assigned partitions when checking
if a topic should be retained from the metadata response when using
RE2J.

Reviewers: David Jacot <djacot@confluent.io>
2025-06-12 10:52:49 -04:00
Gaurav Narula 3c50e23f1f KAFKA-19221 Propagate IOException on LogSegment#close (#19607)
Log segment closure results in right sizing the segment on disk along
with the associated index files.

This is specially important for TimeIndexes where a failure to right
size may eventually cause log roll failures leading to under replication
and log cleaner failures.

This change uses `Utils.closeAll` which propagates exceptions, resulting
in an "unclean" shutdown. That would then cause the broker to attempt to
recover the log segment and the index on next startup, thereby avoiding
the failures described above.

Reviewers: Omnia Ibrahim <o.g.h.ibrahim@gmail.com>, Jun Rao
 <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-11 01:10:37 +08:00
Ken Huang d6861f3f15
MINOR: Use `pollUntilTrue` instead of `waitForCondition` (#19911)
CI / build (push) Waiting to run Details
We can use `pollUntilTrue` instead of `waitForCondition`, thus do a
little refactor to reduce the duplicate code

Reviewers: TengYao Chi <frankvicky@apache.org>, Lan Ding
 <isDing_L@163.com>, TaiJuWu <tjwu1217@gmail.com>
2025-06-09 15:33:00 +08:00
Ken Huang 8fd0d33670
KAFKA-19042 Move PlaintextConsumerSubscriptionTest to client-integration-tests module (#19827)
CI / build (push) Waiting to run Details
Use Java to rewrite PlaintextConsumerSubscriptionTest by new test infra
and move it to client-integration-tests module.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-08 05:06:03 +08:00
Kirk True 861eeb859d
KAFKA-19295: Remove AsyncKafkaConsumer event ID generation (#19915)
CI / build (push) Waiting to run Details
Remove the event IDs from the ApplicationEvent and BackgroundEvent as it
serves no functional purpose other than uniquely identifying events in
the logs.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-07 13:08:22 +01:00
hgh1472 c4a769bc8b
MINOR: Rename ambiguous method name (#19875)
CI / build (push) Waiting to run Details
While reading through the code, I found the method name to be somewhat
ambiguous and not fully descriptive of its purpose.

So I renamed the method to make its purpose clearer and more
self-explanatory. If there was another reason for the original naming,
I’d be happy to hear about it.

Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-06-06 15:03:51 -04:00
PoAn Yang 844b0e651b
KAFKA-19369: Add group.share.assignors config and integration test (#19900)
CI / build (push) Waiting to run Details
* 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>
2025-06-06 14:20:56 +01:00
PoAn Yang e0adec5549
KAFKA-19290: Exploit mapKey optimisation in protocol requests and responses (wip) (#19815)
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>
2025-06-06 14:19:08 +01:00
TaiJuWu f86659423d
KAFKA-19042 Move PlaintextConsumerAssignTest to clients-integration-tests module (#19773)
CI / build (push) Waiting to run Details
The PR do following:
1. rewrite to new test infra
2. rewrite to java
3. move to clients-integration-tests

Reviewers: Ken Huang <s7133700@gmail.com>, Kuan-Po Tseng
<brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-05 23:08:20 +08:00
Matthias J. Sax a662bc5634
MINOR: clean KafkaConsumer tests (#19669)
CI / build (push) Waiting to run Details
- Moving off deprecated methods
- Fixing argument order for assertEquals(...)
- Few other minor cleanups

Reviewers: PoAn Yang <payang@apache.org>, Lianet Magrans
 <lmagrans@confluent.io>, Ken Huang <s7133700@gmail.com>
2025-06-05 06:09:21 -07:00
PoAn Yang 8eb84399f6
MINOR: rackId is Optional#empty if input string is empty (#19906)
Add test case `testRackIdIsEmptyIfValueIsEmptyString`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-05 14:41:40 +08:00
PoAn Yang 949617b0b2
KAFKA-17747: [7/N] Add consumer group integration test for rack aware assignment (#19856)
* Add `RackAwareAssignor`. It uses `racksForPartition` to check the rack
id of a partition and assign it to a member which has the same rack id.
* Add `ConsumerIntegrationTest#testRackAwareAssignment` to check
`racksForPartition` works correctly.

Reviewers: David Jacot <djacot@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-06-04 19:32:17 +02:00
Kirk True 1e917906ab
KAFKA-18573: Add support for OAuth jwt-bearer grant type (#19754)
CI / build (push) Waiting to run Details
Adding support for the `urn:ietf:params:oauth:grant-type:jwt-bearer`
grant type (AKA `jwt-bearer`). Includes further refactoring of the
existing OAuth layer and addition of generic JWT assertion layer that
can be leveraged in the future.

This constitutes the main piece of the JWT Bearer grant type support.

Forthcoming commits/PRs will include improvements for both the
`client_credentials` and `jwt-bearer` grant types in the following
areas:

* Integration test coverage (KAFKA-19153)
* Unit test coverage (KAFKA-19308)
* Top-level documentation (KAFKA-19152)
* Improvements to and documentation for `OAuthCompatibilityTool`
(KAFKA-19307)

Reviewers: Manikumar Reddy <manikumar@confluent.io>, Lianet Magrans
 <lmagrans@confluent.io>

---------

Co-authored-by: Zachary Hamilton <77027819+zacharydhamilton@users.noreply.github.com>
Co-authored-by: Lianet Magrans <98415067+lianetm@users.noreply.github.com>
2025-06-04 09:01:05 -04:00
Xuan-Zhang Gong d783f73288
MINOR: Remove unnecessary checks. (#19891)
The `String.split` method never returns an array containing null
elements.

Reviewers: TengYao Chi <frankvicky@apache.org>, Ken Huang
<s7133700@gmail.com>, Lan Ding <isDing_L@163.com>
2025-06-04 15:33:01 +08:00
Kaushik Raina b1ea280ab1
KAFKA-19250 : txnProducer.abortTransaction() API should not return abortable exception (#19656)
CI / build (push) Waiting to run Details
## Problem
When an `txnProducer.abortTransaction()`operation encounters a
`TRANSACTION_ABORTABLE` error, it currently tries to transition to
`ABORTABLE_ERROR` state. This can create an infinite retry loop since:
1. The abort operation fails with `TRANSACTION_ABORTABLE`
2. We transition to `ABORTABLE_ERROR` state
3. The application recieves instance of TransactionAbortableException
and it retries the abort
4. The cycle repeats

## Solution
For `txnProducer.abortTransaction()`API,  convert
`TRANSACTION_ABORTABLE` errors to fatal errors (`KafkaException`) during
abort operations to ensure clean transaction termination. This prevents
retry loops by:
1. Treating abort failures as fatal errors at application layer
2. Ensuring the transaction can be cleanly terminated
3. Providing clear error messages to the application

## Changes
- Modified `EndTxnHandler.handleResponse()` to convert
`TRANSACTION_ABORTABLE` errors to `KafkaException` during abort
operations
- Set TransactionManager state to FATAL
- Updated test `testAbortableErrorIsConvertedToFatalErrorDuringAbort` to
verify this behavior

## Testing
- Added test case verifying that abort operations convert
`TRANSACTION_ABORTABLE` errors to `KafkaException`
    - Verified that Commit API with TRANSACTION_ABORTABLE error should
set TM to Abortable state
    - Verified that Abort API with TRANSACTION_ABORTABLE error should
convert to Fatal error i.e. KafkaException

## Impact
At application layer, this change improves transaction reliability by
preventing infinite retry loops during abort operations.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-06-03 17:27:15 -07:00
Kaushik Raina 8c71ab03b5
KAFKA-19176: Update Transactional producer to translate retriable into abortable exceptions (#19522)
CI / build (push) Waiting to run Details
### Problem
- Currently, when a transactional producer encounters retriable errors
(like `COORDINATOR_LOAD_IN_PROGRESS`) and exhausts all retries, finally
returns retriable error to Application Layer.
- Application reties can cause duplicate records. As a fix we are
transitioning all retriable errors  as Abortable Error in transaction
producer path.

- Additionally added InvalidTxnStateException as part of
https://issues.apache.org/jira/browse/KAFKA-19177

### Solution
- Modified the TransactionManager to automatically transition retriable
errors to abortable errors after all retries are exhausted. This ensures
that applications can abort transaction when they encounter
`TransactionAbortableException`

- `RefreshRetriableException` like `CoordinatorNotAvailableException`
will be refreshed internally

[[code](6c26595ce3/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java (L1702-L1705))]
till reties are expired, then it will be treated as retriable errors and
translated to `TransactionAbortableException`

- Similarly for InvalidTxnStateException

### Testing
Added test `testSenderShouldTransitionToAbortableAfterRetriesExhausted`
to verify in sender thread:
- Retriable errors are properly converted to abortable state after
retries
- Transaction state transitions correctly and subsequent operations fail
appropriately with TransactionAbortableException

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-06-03 10:21:22 -07:00
Kaushik Raina 82ea9d0fce
MINOR : Handle error for client telemetry push (#19881)
Update catch to handle compression errors

Before :

![image](https://github.com/user-attachments/assets/c5ca121e-ba0c-4664-91f1-20b54abf67cc)

After
```
Sent message: KR Message 376
[kafka-producer-network-thread | kr-kafka-producer] INFO
org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter -
KR: Failed to compress telemetry payload for compression: zstd, sending
uncompressed data
Sent message: KR Message 377
```

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-03 14:29:44 +01:00
Sushant Mahajan df93571f50
KAFKA-19338: Error on read/write of uninitialized share part. (#19861)
- Currently, read and write share state requests were allowed on
uninitialized share partitions (share partitions on which
initializeState has NOT been called). This should not be the case.
- This PR addresses the concern by adding error checks on read and
write. Other requests are allowed (initialize, readSummary, alter).
- Refactored `ShareCoordinatorShardTest` to reduce redundancy and added
some new tests.
- Some request/response classes have also been reformatted.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-06-03 11:26:38 +01:00
Nick Guo 32903a1873
KAFKA-19349 Move CreateTopicsRequestWithPolicyTest to clients-integration-tests (#19849)
Move CreateTopicsRequestWithPolicyTest to clients-integration-tests.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-03 02:52:07 +08:00
Ming-Yen Chung 6826f45fd8
KAFKA-19352 Create offsets topic to fix flaky testCommitAsyncCompletedBeforeConsumerCloses (#19873)
The flakiness occurs when the offsets topic does not yet exist. Hence,
the issue is mitigated by creating the offsets topic in `setup()`. This
serves as a workaround.  The root cause is tracked in
[KAFKA-19357](https://issues.apache.org/jira/browse/KAFKA-19357).

I ran the test 100 times on my Mac and all of them passed.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-02 01:01:24 +08:00
Hong-Yi Chen 77be6f2d74
KAFKA-19053 Remove FetchResponse#of which is not used in production … (#19327)
Removed the unused FetchResponse#of that is not used in production. The
test cases that originally invoked this method have been updated to call
the other

[FetchResponse#of](6af849f864/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java (L232)),
which is currently used by ```KafkaApis```, to maintain the integrity of
the tests.

Reviewers: Jun Rao <junrao@gmail.com>, PoAn Yang <payang@apache.org>,
 Chia-Ping Tsai <chia7712@gmail.com>
2025-06-02 00:48:53 +08:00
Logan Zhu 4eac6adf91
KAFKA-19284 Add documentation to clarify the behavior of null values for all partitionsToOffsetAndMetadata methods. (#19728)
CI / build (push) Waiting to run Details
Adds missing documentation to the `partitionsToOffsetAndMetadata`
methods in both `ListStreamsGroupOffsetsResult` and
`ListShareGroupOffsetsResult` classes to clarify the behavior when a
group does not have a committed offset for a specific partition.

As document in ListConsumerGroupOffsetsResult: > If the group doesn’t
have a committed offset for a specific partition, the corresponding
value in the returned map will be null.

This important detail was previously missing in the JavaDoc of the
stream and share group variants.

Reviewers: Nick Guo <lansg0504@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-01 02:17:47 +08:00
Ming-Yen Chung f9df9b5652
KAFKA-19311 Document commitAsync behavioral differences between Classic and Async Consumer (#19864)
#15613 ensures that all `commitAsync` callbacks are triggered before
`commitSync` completes for `AsyncKafkaConsumer`. However, the related
changes to `ClassicKafkaConsumer`, #15693, were not merged. I assume
this might be because we intend to gradually move toward using AsyncConsumer
instead.

In short, this behavioral difference should be documented.

Reviewers: Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-06-01 01:58:26 +08:00
Nick Guo cc0f06554b
KAFKA-19042 Move GroupAuthorizerIntegrationTest to clients-integration-tests module (#19685)
CI / build (push) Waiting to run Details
move GroupAuthorizerIntegrationTest to clients-integration-tests module

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, keemsisi <keemsisi@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-31 02:34:56 +08:00
Ken Huang 5e601b2b26
KAFKA-19042 Move BaseConsumerTest, SaslPlainPlaintextConsumerTest to client-integration-tests module (#19651)
Use Java to rewrite BaseConsumerTest, SaslPlainPlaintextConsumerTest by
new test infra and move it to client-integration-tests module, the
BaseConsumerTest test is still used, thus we should not remove now.

Reviewers: PoAn Yang <payang@apache.org>, TengYao Chi
<frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-31 02:19:26 +08:00
Lan Ding 43f603cfb7
KAFKA-19351: AsyncConsumer#commitAsync should copy the input offsets (#19855)
`AsyncConsumer#commitAsync` and `AsyncConsumer#commitSync` should copy
the input offsets.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-30 09:36:38 +01:00
Nick Guo a122ac9d51
KAFKA-19042 move ConsumerWithLegacyMessageFormatIntegrationTest to clients-integration-tests module (#19810)
CI / build (push) Waiting to run Details
This PR rewrites `ConsumerWithLegacyMessageFormatIntegrationTest.scala`
in Java and moves it to the `clients-integration-tests module`.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai
 <chia7712@gmail.com>
2025-05-30 01:48:03 +08:00
PoAn Yang d1f1e5c8fd
KAFKA-18904: kafka-configs.sh return resource doesn't exist message [3/N] (#19808)
* Return resource doesn't exist message when users try to describe a
non-existent resource in kafka-configs.sh and kafka-client-metrics.sh.
* For groups type, the command checks both existent groups and
non-existent groups but having dynamic config. If it cannot find a group
in both conditions, return resource doesn't exist message.

Reviewers: Lan Ding <53332773+DL1231@users.noreply.github.com>, Andrew
Schofield <aschofield@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-05-29 18:15:31 +01:00
Ritika Reddy f42abe6db8
KAFKA-19082:[4/4] Complete Txn Client Side Changes (KIP-939) (#19714)
public void completeTransaction(PreparedTxnState preparedTxnState)

The method compares the currently prepared transaction state and the
state passed in the argument.

1.  Commit if the state matches
2. Abort the transaction otherwise.

If the producer is not in a prepared state (i.e., neither
prepareTransaction was called nor initTransaction(true) was called), we
return an INVALID_TXN_STATE error.

Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits
 <alivshits@confluent.io>
2025-05-29 09:06:57 -07:00
Andrew Schofield bd939f56de
MINOR: Removed stale comment (#19848)
Removed a trivial stale comment.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>
2025-05-29 13:44:11 +01:00
Lucas Brutschy 0c116c9629
KAFKA-19335: Membership managers send negative epoch in JOINING (#19818)
There is a sequence of interactions with the membership managers of
KIP-848, KIP-932, KIP-1071 that can put the membership manager into
JOINING state, but where member epoch is set to -1. This can result in
an invalid request being sent, since joining heartbeats should not have
member epoch -1. This may lead to the member failing to join. In the
case of streams, the group coordinator will return INVALID_REQUEST.

This is the sequence triggering the bug, which seems to relatively
likely, caused by two heartbeat responses being received after the next
one has been sent.

```
membershipManager.leaveGroup();
    -> transitions to LEAVING
membershipManager.onHeartbeatRequestGenerated();
    -> transitions to UNSUBSCRIBED
membershipManager.onHeartbeatSuccess(... with member epoch > 0);
    -> unblocks the consumer
membershipManager.onSubscriptionUpdated();
membershipManager.onConsumerPoll();
    -> transitions to JOINING
membershipManager.onHeartbeatSuccess(... with member epoch < 0);
    -> updates the epoch to a negative value
```
Now we are in state `JOINING` with `memberEpoch=-1`, and the next
heartbeat we send will be malformed, triggering `INVALID_REQUEST`

The bug may also be triggered if the `unsubscribe` times out, but this
seems more of a corner case.

To prevent the bug, we are taking two measures: The likely path to
triggering the bug can be prevented by not unblocking an `unsubscribe`
call in the consumer when a non-leave-heartbeat epoch is received. Once
we have sent out leave group heartbeat, we will ignore all heartbeats,
except for those containing `memberEpoch < 0`.

For extra measure, we also prevent the second case (`unsubscribe` timing
out). In this case, the consumer gets unblocked before we have received
the leave group heartbeat response, and may resubscribe to the group. In
this case, we shall just ignore the heartbeat response that contains a
member epoch < 0, once it arrives, and we have already left the
`UNSUBSCRIBED` state.

Reviewers: Lianet Magrans <lmagrans@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>, Shivsundar R <shr@confluent.io>
2025-05-28 16:47:00 +02:00
S.Y. Wang 543fb6c848
KAFKA-19336 Upgrade Jackson to 2.19.0 (#19835)
`JsonNode.fields()` method has been deprecated by 
- https://github.com/FasterXML/jackson-databind/issues/4863
- https://github.com/FasterXML/jackson-databind/pull/4871

So modified accordingly.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-05-28 20:53:43 +08:00
Alieh Saeedi a3d5ca07f8
MINOR: Change `Streams group` to `streams group` (#19813)
As of https://github.com/apache/kafka/pull/19758#discussion_r2097734386,
the capitalization across all messages are aligned.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>, Andrew Schofield
 <aschofield@confluent.io>
2025-05-28 09:56:29 +02:00
Ritika Reddy 17226faa80
MINOR: Fix initProducerId throttle time to non-zero (#19801)
CI / build (push) Waiting to run Details
There was a small error in setting the throttle time in the
InitProducerId response in this PR

[53e95ffcdb (diff-1f7f71b4fbeaf65123e65cf2e4d9c2ae8153820869eb5e88279df979130929d2R69)](url),
which was supposed to be mostly mechanical changes.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-05-27 14:57:23 -07:00
Abhinav Dixit d9233d2f16
MINOR: Remove flakiness caused when producing aborted transaction in ShareConsumerTest (#19824)
This PR attempts to removed the flakiness in
`testAlterReadCommittedToReadUncommittedIsolationLevelWithReleaseAck`
and `testAlterReadCommittedToReadUncommittedIsolationLevelWithRejectAck`.
This flakiness could potentially be caused because we were not ensuring
that the aborted transaction record produce happened. In this PR, I have
added a blocking call to make sure the produce future completes before
we abort the transaction.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-05-27 12:52:07 +01:00
Ming-Yen Chung 0013d78298
KAFKA-19330 Change MockSerializer/Deserializer to use String serializer instead of byte[] (#19812)
While rewriting `EndToEndClusterIdTest` in Java (#19741 ), I found that
the test uses `MockInterceptor` and `MockSerializer` together. However,
`MockSerializer` was using a `byte[]` serializer, while
`MockInterceptor` expected a `String` serializer, leading to a
`ClassCastException`.

I chose to update `MockSerializer` to use `String`, as it is used less
frequently than the `MockInterceptor`. Using `String` also simplifies
the code  by avoiding the need to write expressions like
`"value".getBytes`.

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
 <frankvicky@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-27 18:20:34 +08:00
Nick Guo 0600abdde3
KAFKA-19300 AsyncConsumer#unsubscribe always timeout due to GroupAuthorizationException (#19779)
I verified the behavior by rewriting the
`GroupAuthorizerIntegrationTest` in Java in this PR:
https://github.com/apache/kafka/pull/19685  The state is now correct.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-05-27 00:52:56 +08:00