Commit Graph

600 Commits

Author SHA1 Message Date
Andrew Schofield 21a080f08c
KAFKA-16894: Define feature to enable share groups (#19293)
This PR proposes a switch to enable share groups for 4.1 (preview) and
4.2 (GA).

* `share.version=1` to indicate that share groups are enabled. This is
used as the switch for turning share groups on and off.

In 4.1, the default will be `share.version=0`. Then a user wanting to
evaluate the preview of KIP-932 would use `bin/kafka-features.sh
--bootstrap.server xxxx upgrade --feature share.version=1`.

In 4.2, the default will be `share.version=1`.

Reviewers: Jun Rao <junrao@gmail.com>
2025-04-11 12:14:38 +01:00
Alieh Saeedi 3e0276ebb7
KAFKA-19127: Integration test for altering and describing streams group configs (#19436)
This PR introduces integration tests for verifying
- altering configs
- `describe` output

for a streams group.  The configs have been defined in `KIP-1071`.

Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
2025-04-11 12:48:02 +02:00
Lucas Brutschy 3623726c76
KAFKA-18286: Implement support for streams groups in kafka-groups.sh (#19423)
Add support for streams groups in kafka-groups.sh.

The change adds command-line options `--streams` to list only streams
groups, and value `--group-type streams`. Those two options are mutually
exclusive with other group type and protocol filters specified on the
command line.

Includes a small integration test that spins up a kafka streams
application and lists the group.

Reviewers: Bill Bejeck <bbejeck@apache.org>, Alieh Saeedii
<asaeedi@confluent.io>
2025-04-11 11:17:27 +02:00
Andrew Schofield 99f871a1ef
MINOR: Consolidated message formatter for share group records (#19435)
Create a single formatter for use with `kafka-console-consumer.sh` that
formats all record types for share groups on the `__consumer_offsets`
topic.
2025-04-11 09:46:11 +01:00
Sushant Mahajan c3b7aa6e64
KAFKA-18170: Add create and write timestamp fields in share snapshot [1/N] (#19432)
* We wish to track the time of creation of the `ShareSnapshot` records
so that automated jobs could force their creation if a share partition
has gone cold (no updates for a specified time interval).
* To accomplish this, we have added 2 new fields `CreateTimestamp` and
`WriteTimestamp` in the `ShareSnapshot` record.
* The former tracks snapshot creation due to regular RPC calls while the
latter will track snapshots created by periodic jobs.
* In this PR we have made the requisite changes.
* This is a first of a series of PRs to create the automated jobs and
associated scaffolding.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-04-10 15:56:58 +01:00
Alieh Saeedi fa62bce63d
KAFKA-18287: Add support for kafka-streams-groups.sh --list (#19422)
Implement the core of kafka-streams-groups.sh for `KIP-1071`
- Implement `--list` and its options: (only `--state`)

Reviewers: Bruno Cadonna <cadonna@apache.org>
2025-04-09 17:57:34 +02:00
Andrew Schofield d4d9f11816
KAFKA-18761: [2/N] List share group offsets with state and auth (#19328)
This PR approaches completion of Admin.listShareGroupOffsets() and
kafka-share-groups.sh --describe --offsets.

Prior to this patch, kafka-share-groups.sh was only able to describe the
offsets for partitions which were assigned to active members. Now, the
Admin.listShareGroupOffsets() uses the persister's knowledge of the
share-partitions which have initialised state. Then, it uses this list
to obtain a complete set of offset information.

The PR also implements the topic-based authorisation checking. If
Admin.listShareGroupOffsets() is called with a list of topic-partitions
specified, the authz checking is performed on the supplied list,
returning errors for any topics to which the client is not authorised.
If Admin.listShareGroupOffsets() is called without a list of
topic-partitions specified, the list of topics is discovered from the
persister as described above, and then the response is filtered down to
only show the topics to which the client is authorised. This is
consistent with other similar RPCs in the Kafka protocol, such as
OffsetFetch.

Reviewers: David Arthur <mumrah@gmail.com>, Sushant Mahajan <smahajan@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-04-04 13:25:19 +01:00
Ritika Reddy eeffd8c475
KAFKA-19003: Add forceTerminateTransaction command to CLI tools (#19276)
This patch is part of KIP-939 [Support Participation in
2PC](https://cwiki.apache.org/confluence/display/KAFKA/KIP-939%3A+Support+Participation+in+2PC)

The kafka-transactions.sh tool will support a new command
--forceTerminateTransaction It has one required argument
--transactionalId that would take the transactional id for the
transaction to be terminated.

The command uses the existing Admin#fenceProducers method to forcefully
abort the transaction associated with the specified transactional ID.
Under the hood, it sends an InitProducerId request to the transaction
coordinator with the given transactional ID and keepPreparedTxn = false
by default. This is aligned with the functionality outlined in the KIP.

We will be creating a new public method in the Admin Client **public
TerminateTransactionResult forceTerminateTransaction(String
transactionalId)**, and re-use the existing fence producer method.

Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>
2025-04-02 11:51:26 -07:00
Kuan-Po Tseng 5982943267
KAFKA-18409: ShareGroupStateMessageFormatter should use CoordinatorRecordMessageFormatter (#18510)
ShareGroupStateMessageFormatter should extend
CoordinatorRecordMessageFormatter in order to have a consistent handling
of records of coordinators.

Reviewers: Ken Huang <s7133700@gmail.com>, David Jacot <djacot@confluent.io>
2025-03-31 07:45:00 -07:00
PoAn Yang 4a5ae144ea
KAFKA-19032 Remove TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames (#19270)
The zookeeper mode was removed in 4.0. The test cases don't need to
specify quorum. Following variable and functions can be replaced:
- TestWithParameterizedQuorumAndGroupProtocolNames
- getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly
- getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly
- getTestQuorumAndGroupProtocolParametersAll

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-30 02:11:07 +08:00
Sushant Mahajan eb88e78373
KAFKA-18827: Initialize share group state group coordinator impl. [3/N] (#19026)
* This PR adds impl for the initialize share groups call from the Group
Coordinator perspective.
* The initialize call on persister instance will be invoked by the
`GroupCoordinatorService`, based on the response of the
`GroupCoordinatorShard.shareGroupHeartbeat`. If there is new topic
subscription or member assignment change (topic paritions incremented),
the delta share partitions corresponding to the share group in question
are returned as an optional initialize request.
* The request is then sent to the share coordinator as an encapsulated
timer task because we want the heartbeat response to go asynchronously.
* Tests have been added for `GroupCoordinatorService` and
`GroupMetadataManager`. Existing tests have also been updated.
* A new formatter `ShareGroupStatePartitionMetadataFormatter` has been
added for debugging.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-26 19:40:23 +00:00
David Jacot b6adec48c5
KAFKA-18616; Refactor Tools's ApiMessageFormatter (#18695)
This patch refactors the `ApiMessageFormatter` to follow what we have
done in https://github.com/apache/kafka/pull/18688.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-26 03:12:38 -07:00
TengYao Chi 20bad6efb3
KAFKA-18576 Convert ConfigType to Enum (#18711)
JIRA: KAFKA-18576
After removing ZooKeeper, we no longer need to exclude `client_metrics`
and `group` from `ConfigType#ALL`.

Since it's a common pattern to provide a mechanism to know all values in
enumeration ( Java enum provides ootb), we should convert ConfigType to
enum.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-25 01:10:59 +08:00
Parker Chang b7728d904d
KAFKA-18984: Reset interval.ms By Using kafka-client-metrics.sh (#19213)
kafka-client-metrics.sh cannot reset the interval using `--interval=`.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-24 16:23:49 +05:30
PoAn Yang d497250c22
KAFKA-18999 Remove BrokerMetadata (#19227)
* Replace `BrokerMetadata` with `UsableBroker` in KRaftMetadataCache and
ReassignPartitionsCommand.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-22 19:30:28 +08:00
PoAn Yang 1260b6dbbe
KAFKA-17806 remove this-escape suppress warnings in AclCommand (#19256)
Change AclCommandOptions to final class, so we can remove
`@SuppressWarnings("this-escape")` on it.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-22 02:14:32 +08:00
ClarkChen ec3c319c35
MINOR: Rewrite unchecked operations in Mock API (#19071)
We encountered unchecked or unsafe operations in
`GroupMetadataManagerTest.java`, `KTableImplTest.java`, and
`ConfigCommandIntegrationTest.java`.

* Rewrite getArgument of invocation in InvocationOnMock API because the
implementation of InvocationOnMock discards type anyway in in
`GroupMetadataManagerTest.java`.

* Remove unchecked annotations for using mock API without variable
assignment in `KTableImplTest.java`.

<img width="1422" alt="Screenshot 2025-03-02 at 8 50 55 AM"
src="https://github.com/user-attachments/assets/10ff1799-ebaa-499c-9acd-ca3b30484e6d"
/>
  
* Follow-up: https://github.com/mockito/mockito/issues/1609

Update on March 2.
* Fix unchecked cast for KTableImpl in `KTableImplTest.java`.
<img width="1259" alt="Screenshot 2025-03-02 at 5 17 47 PM"
src="https://github.com/user-attachments/assets/a5ffa3d7-4897-43ee-9b5f-26337e2560c5"
/>

Update on March 10.
* Use anyMap instead any for unchecked map type issues.
<img width="1691" alt="Screenshot 2025-03-10 at 9 36 38 AM"
src="https://github.com/user-attachments/assets/9aabc595-e7ba-4e04-81f6-f238d42af5a6"
/>

Pass all testing.
<img width="946" alt="Screenshot 2025-03-10 at 10 10 56 AM"
src="https://github.com/user-attachments/assets/793f67ea-09dc-44af-9d6c-de15531e9e72"
/>

Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-18 00:47:34 +08:00
PoAn Yang da46cf6e79
KAFKA-17565 Move MetadataCache interface to metadata module (#18801)
### Changes

* Move MetadataCache interface to metadata module and change Scala
function to Java.
* Remove functions `getTopicPartitions`, `getAliveBrokers`,
`topicNamesToIds`, `topicIdInfo`, and `getClusterMetadata` from
MetadataCache interface, because these functions are only used in test
code.

### Performance

* ReplicaFetcherThreadBenchmark
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.fetcher.ReplicaFetcherThreadBenchmark
  ```
  * trunk
  ```
Benchmark (partitionCount) Mode Cnt Score Error Units
ReplicaFetcherThreadBenchmark.testFetcher 100 avgt 2 4775.490 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 500 avgt 2 25730.790 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 1000 avgt 2 55334.206 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 5000 avgt 2 488427.547 ns/op
  ```
  * branch
  ```
Benchmark (partitionCount) Mode Cnt Score Error Units
ReplicaFetcherThreadBenchmark.testFetcher 100 avgt 2 4825.219 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 500 avgt 2 25985.662 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 1000 avgt 2 56056.005 ns/op
ReplicaFetcherThreadBenchmark.testFetcher 5000 avgt 2 497138.573 ns/op
  ```

* KRaftMetadataRequestBenchmark
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.metadata.KRaftMetadataRequestBenchmark
  ```
  * trunk
  ```
Benchmark (partitionCount) (topicCount) Mode Cnt Score Error Units
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 500
avgt 2 884933.558 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 1000
avgt 2 1910054.621 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 5000
avgt 2 21778869.337 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 500
avgt 2 1537550.670 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 1000
avgt 2 3168237.805 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 5000
avgt 2 29699652.466 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 500
avgt 2 3501483.852 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 1000
avgt 2 7405481.182 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 5000
avgt 2 55839670.124 ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 500 avgt 2 333.667
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 1000 avgt 2 339.685
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 5000 avgt 2 334.293
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 500 avgt 2 329.899
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 1000 avgt 2 347.537
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 5000 avgt 2 332.781
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 500 avgt 2 327.085
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 1000 avgt 2 325.206
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 5000 avgt 2 316.758
ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 500 avgt 2 7.569 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 1000 avgt 2 7.565 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 5000 avgt 2 7.574 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 500 avgt 2 7.568 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 1000 avgt 2 7.557 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 5000 avgt 2 7.585 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 500 avgt 2 7.560 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 1000 avgt 2 7.554 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 5000 avgt 2 7.574 ns/op
  ```
  * branch
  ```
Benchmark (partitionCount) (topicCount) Mode Cnt Score Error Units
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 500
avgt 2 910337.770 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 1000
avgt 2 1902351.360 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 10 5000
avgt 2 22215893.338 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 500
avgt 2 1572683.875 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 1000
avgt 2 3188560.081 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 20 5000
avgt 2 29984751.632 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 500
avgt 2 3413567.549 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 1000
avgt 2 7303174.254 ns/op
KRaftMetadataRequestBenchmark.testMetadataRequestForAllTopics 50 5000
avgt 2 54293721.640 ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 500 avgt 2 318.335
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 1000 avgt 2 331.386
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 10 5000 avgt 2 332.944
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 500 avgt 2 340.322
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 1000 avgt 2 330.294
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 20 5000 avgt 2 342.154
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 500 avgt 2 341.053
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 1000 avgt 2 335.458
ns/op
KRaftMetadataRequestBenchmark.testRequestToJson 50 5000 avgt 2 322.050
ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 500 avgt 2 7.538 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 1000 avgt 2 7.548 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 10 5000 avgt 2 7.545 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 500 avgt 2 7.597 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 1000 avgt 2 7.567 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 20 5000 avgt 2 7.558 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 500 avgt 2 7.559 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 1000 avgt 2 7.615 ns/op
KRaftMetadataRequestBenchmark.testTopicIdInfo 50 5000 avgt 2 7.562 ns/op
  ```

* PartitionMakeFollowerBenchmark
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.partition.PartitionMakeFollowerBenchmark
  ```
  * trunk
  ```
Benchmark Mode Cnt Score Error Units
PartitionMakeFollowerBenchmark.testMakeFollower avgt 2 158.816 ns/op
  ```
  * branch
  ```
Benchmark Mode Cnt Score Error Units
PartitionMakeFollowerBenchmark.testMakeFollower avgt 2 160.533 ns/op
  ```

* UpdateFollowerFetchStateBenchmark
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.partition.UpdateFollowerFetchStateBenchmark
  ```
  * trunk
  ```
Benchmark Mode Cnt Score Error Units
UpdateFollowerFetchStateBenchmark.updateFollowerFetchStateBench avgt 2
4975.261 ns/op
UpdateFollowerFetchStateBenchmark.updateFollowerFetchStateBenchNoChange
avgt 2 4880.880 ns/op
  ```
  * branch
  ```
Benchmark Mode Cnt Score Error Units
UpdateFollowerFetchStateBenchmark.updateFollowerFetchStateBench avgt 2
5020.722 ns/op
UpdateFollowerFetchStateBenchmark.updateFollowerFetchStateBenchNoChange
avgt 2 4878.855 ns/op
  ```


* CheckpointBench
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.server.CheckpointBench
  ```
  * trunk
  ```
Benchmark (numPartitions) (numTopics) Mode Cnt Score Error Units
CheckpointBench.measureCheckpointHighWatermarks 3 100 thrpt 2 0.997
ops/ms
CheckpointBench.measureCheckpointHighWatermarks 3 1000 thrpt 2 0.703
ops/ms
CheckpointBench.measureCheckpointHighWatermarks 3 2000 thrpt 2 0.486
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 100 thrpt 2 1.038
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 1000 thrpt 2 0.734
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 2000 thrpt 2 0.637
ops/ms
  ```
  * branch
  ```
Benchmark (numPartitions) (numTopics) Mode Cnt Score Error Units
CheckpointBench.measureCheckpointHighWatermarks 3 100 thrpt 2 0.990
ops/ms
CheckpointBench.measureCheckpointHighWatermarks 3 1000 thrpt 2 0.659
ops/ms
CheckpointBench.measureCheckpointHighWatermarks 3 2000 thrpt 2 0.508
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 100 thrpt 2 0.923
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 1000 thrpt 2 0.736
ops/ms
CheckpointBench.measureCheckpointLogStartOffsets 3 2000 thrpt 2 0.637
ops/ms
  ```

* PartitionCreationBench
  ```
./jmh-benchmarks/jmh.sh -f 1 -i 2 -wi 2
org.apache.kafka.jmh.server.PartitionCreationBench
  ```
  * trunk
  ```
Benchmark (numPartitions) (useTopicIds) Mode Cnt Score Error Units
PartitionCreationBench.makeFollower 20 false avgt 2 5.997 ms/op
PartitionCreationBench.makeFollower 20 true avgt 2 6.961 ms/op
  ```
  * branch
  ```
Benchmark (numPartitions) (useTopicIds) Mode Cnt Score Error Units
PartitionCreationBench.makeFollower 20 false avgt 2 6.212 ms/op
PartitionCreationBench.makeFollower 20 true avgt 2 7.005 ms/op
  ```

Reviewers: Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-17 23:59:11 +08:00
Andrew Schofield 5e7445a6d6
KAFKA-17516 Synonyms for client metrics configs (#17264)
This PR brings client metrics configuration resources in line with the
other config resources in terms of handling synonyms and defaults.
Specifically, configs which are not explicitly set take their hard-coded
default values, and these are reported by `kafka-configs.sh --describe`
and `Kafka-client-metrics.sh --describe`. Previously, they were omitted
which means the administrator needed to know the default values.

The ConfigHelper was changed so that the handling of client metrics
configuration matches that of group configuration.

Reviewers: poorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-14 16:05:40 +08:00
Mickael Maison 759fbbba8b
KAFKA-14484: Move UnifiedLog to storage module (#19030)
Rewrite UnifiedLog in Java

Reviewers: Jun Rao <jun@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-13 10:49:55 +01:00
Andrew Schofield 278a93c45d
KAFKA-18901: [1/N] Improved homogeneous SimpleAssignor (#19142)
The current homogeneous SimpleAssignor for share groups is not very good
at revoking partitions which have previously been assigned when the
number of members increases. This PR improves the situation.

It also fixes the sorting of assignments in `kafka-consumer-groups.sh`
and `kafka-share-groups.sh` so that it sorts partition indices
numerically instead of alphabetically. It also adds the missing number
of partitions column for share groups.
2025-03-11 10:08:31 +00:00
Lucas Brutschy fc2e3dfce9
MINOR: Disallow unused local variables (#18963)
Recently, we found a regression that could have been detected by static
analysis, since a local variable wasn't being passed to a method during
a refactoring, and was left unused. It was fixed in
[7a749b5](7a749b589f),
but almost slipped into 4.0. Unused variables are typically detected by
IDEs, but this is insufficient to prevent these kinds of bugs. This
change enables unused local variable detection in checkstyle for Kafka.

A few notes on the usage:
- There are two situations in which people actually want to have a local
variable but not use it. First, there are `for (Type ignored:
collection)` loops which have to loop `collection.length` number of
times, but that do not use `ignored` in the loop body. These are
typically still easier to read than a classical `for` loop. Second, some
IDEs detect it if a return value of a function such as `File.delete` is
not being used. In this case, people sometimes store the result in an
unused local variable to make ignoring the return value explicit and to
avoid the squiggly lines.
- In Java 22, unsued local variables can be omitted by using a single
underscore `_`. This is supported by checkstyle. In pre-22 versions,
IntelliJ allows such variables to be named `ignored` to suppress the
unused local variable warning. This pattern is often (but not
consistently) used in the Kafka codebase. This is, however, not
supported by checkstyle.

Since we cannot switch to Java 22, yet, and we want to use automated
detection using checkstyle, we have to resort to prefixing the unused
local variables with `@SuppressWarnings("UnusedLocalVariable")`. We have
to apply this in 11 cases across the Kafka codebase. While not being
pretty, I'd argue it's worth it to prevent bugs like the one fixed in
[7a749b5](7a749b589f).

Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur
<mumrah@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Bruno
Cadonna <cadonna@apache.org>, Kirk True <ktrue@confluent.io>
2025-03-10 09:37:35 +01:00
Ken Huang d5413fdb48
KAFKA-17856 Move ConfigCommandTest and ConfigCommandIntegrationTest to tool module (#17767)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-09 21:05:36 +08:00
Apoorv Mittal 8b955b54da
MINOR: Wrapping exceptions in usage for groups and metrics utility (#19151)
If specified an invalid option then an exception trace appears with
`kafka-client-metrics.sh` and `kafka-groups.sh` utilities. Then once has
to explicitly remove the invalid argument and append `--help` to fetch
correct options. The PR fixes below error message to one with `cause`
and `usage`. This behaviour is similar to `kafka-console-consumer.sh`
and `kafka-console-share-consumer.sh`

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-07 16:30:16 +00:00
Chirag Wadhwa 899cdb598a
KAFKA-18811: Added command configs to admin client as well in VerifiableShareConsumer (#19130)
This PR includes a new flag in VerifiableShareConsumer.java called
command.config to include a properties file for admin client
related configs

Co-authored-by: Andrew Schofield <aschofield@confluent.io>

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-03-06 13:27:15 +00:00
Kuan-Po Tseng cbd72cc216
KAFKA-14121: AlterPartitionReassignments API should allow callers to specify the option of preserving the replication factor (#18983)
Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi <kitingiao@gmail.com>
2025-03-05 11:23:12 +00:00
TengYao Chi e0c77140b2
KAFKA-17039 KIP-919 supports for unregisterBroker (#19063)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-03-01 23:55:35 +08:00
Sushant Mahajan 3fc103b48b
KAFKA-18629: ShareGroupDeleteState admin client impl. (#18928)
* In this PR, we add various infra classes needed to support the
`deleteShareGroups` functionality via the `kafka-share-groups.sh`
script, as well as the implementation of `kafka-share-groups.sh --delete`.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-02-22 16:21:10 +00:00
Matthias J. Sax 538a60e1b3
MINOR: disallow rawtypes and fail build (#18877)
Cleanup code to avoid rawtype, and add suppressions where necessary.
Change the build to fail on rawtype warning.

Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-02-19 13:11:49 -08:00
Ismael Juma 3dba3125e9
KAFKA-18601: Assume a baseline of 3.3 for server protocol versions (#18845)
3.3.0 was the first KRaft release that was deemed production-ready and also
when KIP-778 (KRaft to KRaft upgrades) landed. Given that, it's reasonable
for 4.x to only support upgrades from 3.3.0 or newer (the metadata version also
needs to be set to "3.3" or newer before upgrading).

Noteworthy changes:
1. `AlterPartition` no longer includes topic names, which makes it possible to
simplify `AlterParitionManager` logic.
2. Metadata versions older than `IBP_3_3_IV3` have been removed and
`IBP_3_3_IV3` is now the minimum version.
3. `MINIMUM_BOOTSTRAP_VERSION` has been removed.
4. Removed `isLeaderRecoverySupported`, `isNoOpsRecordSupported`,
`isKRaftSupported`, `isBrokerRegistrationChangeRecordSupported` and
`isInControlledShutdownStateSupported` - these are always `true` now.
Also removed related conditional code.
5. Removed default metadata version or metadata version fallbacks in
multiple places - we now fail-fast instead of potentially using an incorrect
metadata version.
6. Update `MetadataBatchLoader.resetToImage` to set `hasSeenRecord`
based on whether image is empty - this was a previously existing issue that
became more apparent after the changes in this PR.
7. Remove `ibp` parameter from `BootstrapDirectory`
8. A number of tests were not useful anymore and have been removed.

I will update the upgrade notes via a separate PR as there are a few things that
need changing and it would be easier to do so that way.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Justine Olshan <jolshan@confluen.io>, Ken Huang <s7133700@gmail.com>
2025-02-19 05:35:42 -08:00
xijiu 1dcdbf78bb
KAFKA-18798 The replica placement policy used by ReassignPartitionsCommand is not aligned with kraft controller (#18914)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-18 16:02:00 +08:00
Jimmy Wang 85c337af44
KAFKA-18755 Align timeout in kafka-share-groups.sh (#18908)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-17 04:48:25 +08:00
Andrew Schofield 79e853d68e
KAFKA-18761: Complete listing of share group offsets [1/N] (#18894)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-14 18:55:20 +00:00
David Jacot aec0e555be
MINOR: Mark IBP_4_0_IV3 as production ready! (#18902)
This patch marks IBP_4_0_IV3 as production ready for the Apache Kafka 4.0 release. It also introduced IBP_4_1_IV0 as the next development version.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-02-14 08:17:11 -08:00
Ismael Juma da166b9dc1
MINOR: Remove 'disable caching' workaround for Java 8 in connect (#18823)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Greg Harris <greg.harris@aiven.io>
2025-02-06 14:24:28 -08:00
Colin Patrick McCabe b2b2408692
KAFKA-18360 Remove zookeeper configurations (#18566)
Remove broker.id.generation.enable and reserved.broker.max.id, which are not used in KRaft mode.
Remove inter.broker.protocol.version, which is not used in KRaft mode.

Reviewers: PoAn Yang <payang@apache.org>, Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-06 22:22:11 +08:00
Sanskar Jhajharia 7dbed2f6e8
[KAFKA-16720] AdminClient Support for ListShareGroupOffsets (2/2) (#18671)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Sushant Mahajan <smahajan@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2025-02-05 14:38:09 +00:00
Ismael Juma 78aff4fede
KAFKA-18659: librdkafka compressed produce fails unless api versions returns produce v0 (#18727)
Return produce v0-v2 as supported versions in `ApiVersionsResponse`, but disable support
for it everywhere else.

Since clients pick the highest supported version by both client and broker during version
negotiation, this solves the problem with minimal tech debt (even though it's not ideal that
`ApiVersionsResponse` becomes inconsistent with the actual protocol support).

Add one test for the socket server handling (in `ProcessorTest`) and one test for the
client behavior (in `ProduceRequestTest`). Adjust a couple of api versions tests to verify
the new behavior.

Finally, include a few clean-ups in `ApiKeys`, `Protocol`, `ProduceRequest`,
`ProduceRequestTest` and `BrokerApiVersionsCommandTest`.

Reference to related librdkafka issue:
https://github.com/confluentinc/librdkafka/issues/4956

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
2025-02-01 16:08:54 -08:00
yx9o c0b5d3334a
MINOR: Improve error message for invalid topic in TopicCommand (#18714)
Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-30 12:07:45 +01:00
TengYao Chi 97a228070e
KAFKA-18619: New consumer topic metadata events should set requireMetadata flag (#18668)
Reviewers: Lianet Magrans <lmagrans@confluent.io>
2025-01-29 08:36:05 -05:00
David Arthur 8c0a0e07ce
KAFKA-17587 Refactor test infrastructure (#18602)
This patch reorganizes our test infrastructure into three Gradle modules:

":test-common:test-common-internal-api" is now a minimal dependency which exposes interfaces and annotations only. It has one project dependency on server-common to expose commonly used data classes (MetadataVersion, Feature, etc). Since this pulls in server-common, this module is Java 17+. It cannot be used by ":clients" or other Java 11 modules.

":test-common:test-common-util" includes the auto-quarantined JUnit extension. The @Flaky annotation has been moved here. Since this module has no project dependencies, we can add it to the Java 11 list so that ":clients" and others can utilize the @Flaky annotation

":test-common:test-common-runtime" now includes all of the test infrastructure code (TestKitNodes, etc). This module carries heavy dependencies (core, etc) and so it should not normally be included as a compile-time dependency.

In addition to this reorganization, this patch leverages JUnit SPI service discovery so that modules can utilize the integration test framework without depending on ":core". This will allow us to start moving integration tests out of core and into the appropriate sub-module. This is done by adding ":test-common:test-common-runtime" as a testRuntimeOnly dependency rather than as a testImplementation dependency. A trivial example was added to QuorumControllerTest to illustrate this.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-24 09:03:43 -05:00
David Jacot 76bf38a4fd
KAFKA-18604; Update transaction coordinator (#18636)
This patch updates the transaction coordinator record to use the new coordinator record definition.

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-21 08:36:23 +01:00
Ismael Juma 87b37a4065
KAFKA-14552: Assume a baseline of 3.0 for server protocol versions (#18497)
Kafka 4.0 will remove support for zk mode and will require conversion to kraft
before upgrading to 4.0. The minimum kraft version is 3.0 (aka 3.0-IV1).

This provides an opportunity to remove exclusively server side protocols versions
that only exist to allow direct upgrades from versions older than 3.0 or that are
used only by zk mode.

Since KRaft became production ready in 3.3, we should consider setting the
baseline to 3.3. But that requires more discussion and it can be done via a
separate change (KAFKA-18601).

Protocol changes:
* Remove RequestHeader v0 (only used by ControlledShutdown v0)
* Remove WriteTxnMarkers v0
* Remove all versions of ControlledShutdown, LeaderAndIsr, StopReplica, UpdateMetadata

In order to remove all versions safely, extend generator to support setting
"versions" to "none". In this case, we no longer generate the `*Data` classes,
but we still reserve the id for the relevant protocol api (so it doesn't get
accidentally used for something else). The protocol documentation is correct
after these changes.

We kept a simplified version of `LeaderAndIsr{Request|Response}` because
it's used by many tests that are still relevant in kraft mode. Once
KAFKA-18486 is done, it may be possible to remove it (I left a comment on
the ticket). Similarly, KAFKA-18487 may make it possible to remove
the introduced `StopReplicaPartitionState` (left a comment on that ticket too).

There are a number of places that were adjusted to include an
`ApiKeys.hasValidVersion` check.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-20 13:51:44 -08:00
TengYao Chi 029d9184c6
KAFKA-18565 Cleanup SaslSetup (#18586)
Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-19 03:14:59 +08:00
Chirag Wadhwa 8cc560e954
Infrastructure for system tests for the new share consumer client (#18209)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-01-17 12:03:32 +00:00
Ken Huang 3c1f965c60
KAFKA-18521 Cleanup NodeApiVersions zkMigrationEnabled field (#18535)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-16 20:05:04 +08:00
Kirk True 45e3c21e9a
KAFKA-17915: Convert Kafka Client system tests to use KRaft (#17669)
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-14 14:47:15 +01:00
Kirk True 403fe71c5f
KAFKA-18400: Don't use YYYY when formatting/parsing dates in Java client (#18404)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-11 18:29:01 +00:00
Ismael Juma d4aee71e36
KAFKA-18465: Remove MetadataVersions older than 3.0-IV1 (#18468)
Apache Kafka 4.0 will only support KRaft and 3.0-IV1 is the minimum version supported by KRaft. So, we can assume that Apache Kafka 4.0 will only communicate with brokers that are 3.0-IV1 or newer.

Note that KRaft was only marked as production-ready in 3.3, so we could go further and set the baseline to 3.3. I think we should have that discussion, but it made sense to start with the non controversial parts.

Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <david.jacot@gmail.com>
2025-01-11 09:42:39 -08:00
Abhinav Dixit b51b31ed9c
KAFKA-18428: Measure share consumers performance (#18415)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-08 14:33:13 +00:00
Peter Lee 0377e807ff
MINOR: Use Producer interface and ClusterInstance producer factory (#18197)
Reviewers: David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 20:53:30 +08:00
Jhen-Yung Hsu f95726a211
KAFKA-18417 Remove controlled.shutdown.max.retries and controlled.shutdown.retry.backoff.ms (#18431)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 17:13:42 +08:00
Ken Huang d874aa42f3
KAFKA-18368 Remove TestUtils#MockZkConnect and remove zkConnect from TestUtils#createBrokerConfig (#18352)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-07 21:03:13 +08:00
NICOLAS GUYOMAR ea3459ee5d
MINOR: Update Consumer group timeout default to 30 sec (#16406)
30 sec is more inline with the default request.timeout.ms and is still very acceptable for a CLI interaction in my opinion

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2025-01-07 18:23:49 +05:30
Matthias J. Sax 738bd928f1
MINOR: cleanup JavaDocs for deprecation warnings (#18402)
Reviewers: Bill Bejeck <bbejeck@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-01-07 09:18:33 +00:00
David Jacot 7b6e94642a
KAFKA-18303; Update ShareCoordinator to use new record format (#18396)
Following https://github.com/apache/kafka/pull/18261, this patch updates the Share Coordinator to use the new record format.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Andrew Schofield <aschofield@confluent.io>
2025-01-06 23:59:07 -08:00
Ismael Juma 409a43eff7
MINOR: Collection/Option usage simplification via methods introduced in Java 9 & 11 (#18305)
Relevant methods:
1. `List.of`, `Set.of`, `Map.of` and similar (introduced in Java 9)
2. Optional: `isEmpty` (introduced in Java 11), `stream` (introduced in Java 9).

Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
2025-01-03 16:13:39 -08:00
Ismael Juma 73ab7ee4ea
MINOR: Use `Files.readString/writeString` and `String.repeat` to simplify code (#18372)
The 3 methods were introduced in Java 11.

Reviewers: Divij Vaidya <diviv@amazon.com>
2025-01-02 17:50:27 -08:00
Ismael Juma d6f24d3665
Use `instanceof` pattern to avoid explicit cast (#18373)
This feature was introduced in Java 16.

Reviewers: David Arthur <mumrah@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-01-02 09:32:51 -08:00
Andrew Schofield 0344f8f5ae
KAFKA-18273: KIP-1099 verbose display share group options (#18259)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2025-01-02 09:12:52 +00:00
PoAn Yang b4be178599
KAFKA-17393: Remove log.message.format.version/message.format.version (KIP-724) (#18267)
Based on [KIP-724](https://cwiki.apache.org/confluence/display/KAFKA/KIP-724%3A+Drop+support+for+message+formats+v0+and+v1), the `log.message.format.version` and `message.format.version` can be removed in 4.0.

These configs effectively a no-op with inter-broker protocol version 3.0 or higher
since Apache Kafka 3.0, so the impact should be minimal.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2024-12-21 15:35:15 -08:00
David Jacot d67379c310
KAFKA-18301; Make coordinator records first class citizen (#18261)
This patch is the first one in a series to improve how coordinator records are managed. It focuses on making coordinator records first class citizen in the generator.
* Introduce `coordinator-key` and `coordinator-value` in the schema;
* Introduce `apiKey` for those. This is done to avoid relying on the version to determine the type.
* It also allows the generator to enforce some rules: the key cannot use flexible versions, the key must have a single version `0`, there must be a key and a value for a given api key, etc.
* It generates an enum with all the coordinator record types. This is pretty handy in the code.

The patch also updates the group coordinators to use those.

Reviewers: Jeff Kim <jeff.kim@confluent.io>, Andrew Schofield <aschofield@confluent.io>
2024-12-20 12:16:14 +01:00
David Jacot b31aa65115
MINOR: Fix MessageFormatters (#18266)
While looking at the message formatters in https://github.com/apache/kafka/pull/18261, I have noticed at few incorrect test cases.
* We should not log anything when the record type is unknown because the formatters have clear goals.
* We should not parse the value when the key is null or when the key cannot be parsed. While it works in the tests, in practice, this is wrong because we cannot assume that type of the value if the type of the key is not defined. The key drives the type of the entire record.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-19 16:12:50 +01:00
Matthias J. Sax a9eb06bbdf
MINOR: improve StreamsResetter logging (#18237)
StreamsResetter should log the deprecation warning only if the deprecated
flag is used.

Reviewers: Bruno Cadonna <bruno@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-18 18:29:42 -08:00
Mickael Maison 57eb5fd7dc
KAFKA-14587: Move AclCommand to tools (#17880)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 20:05:46 +01:00
Lianet Magrans 84bc0c26ee
KAFKA-18224: Explicit group protocol setting in streams resetter (#18172)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-12-13 14:31:50 -05:00
TengYao Chi b37b89c668
KAFKA-9366 Upgrade log4j to log4j2 (#17373)
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations

Co-authored-by: Lee Dongjin <dongjin@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:14:31 +08:00
PoAn Yang d5ad9228cf
KAFKA-17750; Extend kafka-consumer-groups command line tool to support new consumer group (part 3) (#18141)
This patch extends the `kafka-consumer-groups` command line tool to support the new consumer group as described in KIP-1099.

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

Reviewers: David Jacot <djacot@confluent.io>
2024-12-13 01:31:39 -08:00
Gantigmaa Selenge 747dc172e8
KIP-1073: Return fenced brokers in DescribeCluster response (#17524)
mplementation of KIP-1073: Return fenced brokers in DescribeCluster response.
Add new unit and integration tests for describeCluster.

Reviewers: Luke Chen <showuon@gmail.com>
2024-12-13 10:58:11 +08:00
Lianet Magrans 7a64623e40
Set protocol for streams tests (#18160)
Reviewers: Bill Bejeck <bill@confluent.io>
2024-12-12 13:33:43 -05:00
PoAn Yang 156d551603
MINOR: suppress deprecation warnings for MemberDescription (#18139)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-12 01:05:31 +08:00
Kuan-Po Tseng d2ad418cfd
KAFKA-18156 VerifiableConsumer should ignore "--session-timeout" when using CONSUMER protocol (#18036)
Reviewers: TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-11 21:12:46 +08:00
PoAn Yang c8380ae779
KAFKA-17750: Extend kafka-consumer-groups command line tool to support new consumer group (part 2) (#18034)
* Add fields `groupEpoch` and `targetAssignmentEpoch` to `ConsumerGroupDescription.java`.
* Add fields `memberEpoch` and `upgraded` to `MemberDescription.java`.
* Add assertion to `PlaintextAdminIntegrationTest#testDescribeClassicGroups` to make sure member in classic group returns `upgraded` as `Optional.empty`.
* Add new case `testConsumerGroupWithMemberMigration` to `PlaintextAdminIntegrationTest` to make sure migration member has correct `upgraded` value. Add assertion for `groupEpoch`, `targetAssignmentEpoch`, `memberEpoch` as well.

Reviewers: David Jacot <djacot@confluent.io>

Signed-off-by: PoAn Yang <payang@apache.org>
2024-12-10 05:02:20 -08:00
Andrew Schofield e7d986e48c
KAFKA-17550: DescribeGroups v6 exploitation (#17706)
This PR introduces the DescribeGroups v6 API as part of KIP-1043. This adds an error message for the described groups so that it is possible to get some context on the error. It also changes the behaviour for when the group ID cannot be found but returning error code GROUP_ID_NOT_FOUND rather than NONE.

Reviewers: David Jacot <djacot@confluent.io>
2024-12-05 23:12:24 -08:00
Dongnuo Lyu e30edb3eff
KAFKA-18052: Decouple the dependency of feature stable version to the metadata version (#17886)
Currently the validation of feature upgrade relies on the supported version range generated during registration. For a given feature, its max supported feature version in production is set to be the default version value (the latest feature version with bootstrap metadata value smaller or equal to the latest production metadata value).

This patch introduces a LATEST_PRODUCTION value independent from the metadata version to each feature so that the highest supported feature version can be customized by the feature owner.

The change only applies to dynamic feature upgrade. During formatting, we still use the default value associated the metadata version.

Reviewers: Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
2024-12-05 11:07:47 -08:00
David Jacot c3506834e9
KAFKA-17598; Command line validation tool for RE2J regex (#18031)
This patch introduces the `--validate-regex` argument to the `kafka-consumer-group` command line tool as defined in KIP-848. The new argument allows the verification of RE2 regular expressions.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
2024-12-04 23:40:32 -08:00
Ken Huang 2b43c49f51
KAFKA-18050 Upgrade the checkstyle version to 10.20.2 (#17999)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-05 10:59:18 +08:00
Ken Huang 9d23f89e05
KAFKA-17338 ConsumerConfig should prevent using partition assignors with CONSUMER group protocol (#16899)
Reviewers: Kirk True <ktrue@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Lianet Magrans <lmagrans@confluent.io>
2024-11-29 09:36:29 -05:00
Andrew Schofield e7bbcdb251
KAFKA-18090: Add ShareMemberDescription and Assignment (#17975)
Introduce ShareMemberDescription and ShareMemberAssignment as distinct classes for share groups. Although the correspondence with consumer groups is fairly close, the concepts are likely to diverge over time and separating these concepts now makes sense.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2024-11-29 10:20:01 +05:30
TaiJuWu e1ba01d214
KAFKA-18109 Fix failed SaslClientsWithInvalidCredentialsTest in tools module (#17971)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-28 10:59:21 +08:00
Colin Patrick McCabe cdf3aab661
MINOR: remove zk from several tests (#17949)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-28 04:52:04 +08:00
Calvin Liu 2b2b3cd355
KAFKA-18062: use feature version to enable ELR (#17867)
Replace the ELR static config with feature version.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2024-11-26 14:40:23 -08:00
Andrew Schofield d17a149205
KAFKA-17956 Remove Admin.listShareGroups (#17912)
KIP-1043 introduced Admin.listGroups as the way to list all types of groups. As a result, Admin.listShareGroups has been removed. This PR is the final step of the removal.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-25 22:05:35 +08:00
Manikumar Reddy 3268435fd6
KAFKA-18013: Add AutoOffsetResetStrategy internal class (#17858)
- Deprecates OffsetResetStrategy enum
- Adds new internal class AutoOffsetResetStrategy
- Replaces all OffsetResetStrategy enum usages with AutoOffsetResetStrategy
- Deprecate old/Add new constructors to MockConsumer

 Reviewers: Andrew Schofield <aschofield@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2024-11-25 19:11:12 +05:30
PoAn Yang 70babd5716
KAFKA-18079 consumer-config does not work with console-share-consumer (#17925)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-24 20:13:14 +08:00
TaiJuWu acd92be6ea
MINOR: reduce topicCommandTest brokers from 6 to 3 (#17875)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-23 22:22:29 +08:00
PoAn Yang eb641b5b1c
KAFKA-18059 kafka-metadata-quorum.sh add-controller subcommand can't recognize argument --config (#17907)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-22 23:20:23 +08:00
Andrew Schofield 32c887b05e
KAFKA-17949: Introduce GroupState and replace ShareGroupState (#17763)
This PR introduces the unified GroupState enum for all group types from KIP-1043. This PR also removes ShareGroupState and begins the work to replace Admin.listShareGroups with Admin.listGroups. That will complete in a future PR.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2024-11-19 21:17:12 +05:30
PoAn Yang 078d34f39d
KAFKA-17910 Create integration tests for Admin.listGroups and Admin.describeClassicGroups (#17712)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-18 16:35:48 +08:00
Yung e4b8644400
KAFKA-17992 Remove getUnderlying and isKRaftTest from ClusterInstance (#17802)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-14 17:11:19 +08:00
Ken Huang 6bc7be70d7
KAFKA-17922 add helper to ClusterInstance to create client component (#17666)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-13 09:39:15 +08:00
Ken Huang 207b35901c
KAFKA-17314 Fix the typo: `maxlifeTimeMs` (#17749)
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-11-12 16:26:29 +08:00
Kirk True 42ea29c421
KAFKA-17925 Convert Kafka Client integration tests to use KRaft (#17670)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-10 10:38:01 +08:00
TengYao Chi 393455eb1a
KAFKA-17837 Rewrite DeleteTopicTest (#17579)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-10 03:06:52 +08:00
Kuan-Po Tseng e3f953483c
KAFKA-17857 Move AbstractResetIntegrationTest and subclasses to tools (#17594)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-05 04:16:19 +08:00
kevin-wu24 ebb3202e01
KAFKA-16964 Integration tests for adding and removing voters (#17582)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-05 03:09:37 +08:00
Andrew Schofield 3d9f88daf3
KAFKA-17546 Admin.listGroups and kafka-groups.sh (#17626)
This implements the kafka-groups.sh tool and Admin.listGroups method defined in KIP-1043.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-02 05:37:04 +08:00
Dmitry Werner 1eb7644349
KAFKA-16845 Migrate ReplicationQuotasTestRig to new test infra (#17089)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-25 03:33:01 +08:00
TengYao Chi 553e6b4c6d
KAFKA-17860 Remove log4j-appender module (#17588)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-24 18:13:30 +08:00
TaiJuWu 661bed242e
MINOR: add controller-related tests to metadataQuorumCommandTest (#17486)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-24 03:08:26 +08:00
PoAn Yang 2d896d9130
KAFKA-17614: Remove AclAuthorizer (#17424)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-10-23 17:07:48 +02:00
PoAn Yang 54a83c5b9a
KAFKA-17839 DescribeConsumerGroupTest doesn't really test different arguments (#17556)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-23 02:07:33 +08:00
Eric Chang 6b28e81ba1
KAKFA-17173 move quota config params from KafkaConfig to QuotaConfig (#17505)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-19 18:01:06 +08:00
Kuan-Po Tseng 8adfdbbde0
KAFKA-17256 KRAFT should honor the listener name and security protocol from ClusterConfig (#16824)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-17 10:34:15 +08:00
Sushant Mahajan 5545d717c3
KAFKA-17633: Add share group record formatter and parser. (#17467)
As part of KIP-932, a new internal topic __share_group_state was introduced. There are 2 types of records which are currently being added in this topic - ShareSnapshotKey/Value and ShareUpdateKey/Value
In light of this, we must make the existing tooling like kafka-console-consumer and kafka-dump-log aware of these records for debugging and introspection purposes.
This PR introduces ShareGroupStateMessageFormatter to be used used with kafka-console-consumer and adds an internal class ShareGroupStateMessageParser in DumpLogSegments.scala.
Unit tests have been added to DumpLogSegmentsTest.scala


Reviewers:  Andrew Schofield <aschofield@confluent.io>,  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-10-15 11:44:15 +05:30
Ken Huang 203f3237ec
KAFKA-17374 add bootstrap.controller to kafka-reassign-partitions.sh (#16964)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-14 21:17:00 +08:00
Linsiyuan9 76a1af984b
KAFKA-17746 Replace JavaConverters with CollectionConverters (#17451)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-14 17:13:20 +08:00
Gaurav Narula b03fe66cfe
KAFKA-17759 Remove Utils.mkSet (#17460)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-11 21:20:43 +08:00
PoAn Yang 9bbf0950f9
KAFKA-17387 Remove broker-list in VerifiableConsumer (#17406)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-10 11:41:53 +08:00
Mickael Maison 2836f7aaae
KAFKA-14705 Remove deprecated classes and options in tools (#17420)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-09 10:33:34 +08:00
David Arthur 51482fa298
MINOR Consolidate junit-platform.properties (#17399)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-08 10:51:33 -04:00
Mickael Maison e124e06e75
MINOR: Small cleanup in ToolsUtils (#17388)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-08 11:56:39 +08:00
Ken Huang 2805b42543
HOTFIX: fix build error of TopicCommand (#17401)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-08 08:42:43 +08:00
Federico Valeri 8dbbf5ea76
KAFKA-14705: Move topic filters to tools module (#17387)
After MirrorMaker 1 removal, there are no other modules dependencies for these classes, so we can safely move them to tools module.

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-10-07 16:54:36 +02:00
PoAn Yang d0653378bf
KAFKA-17707 Remove zk from BaseConsumerTest (#17383)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-10-07 17:46:46 +08:00
Dmitry Werner 672c617233
KAFKA-14577: Move ConsoleProducer to tools module (#17157)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Federico Valeri <fedevaleri@gmail.com>
2024-10-07 11:19:59 +02:00
Justine Olshan c3f13b5c57
KAFKA-16308 [4/4]: Add release-version flag to upgrade and downgrade commands (#17362)
I've added the release-version flag to the upgrade and downgrade commands. I've also added tests.

While working on this, I realized that we reveal non-production features to be returned in the version-mapping and dependencies commands. I have changed this to only return production features (except in tests) and added tests for this.

Reviewers: Jun Rao <jun@confluent.io>
2024-10-04 13:03:54 -07:00
Federico Valeri d0ad84df5d
MINOR: producer perf improvements (#17348)
Adding some missing input checks and fixing a formatting issue.

Signed-off-by: Federico Valeri <fedevaleri@gmail.com>

Reviewers: Luke Chen <showuon@gmail.com>
2024-10-03 10:29:19 +08:00
Chia-Ping Tsai 979740b49d
KAFKA-17589 Move JUnit extensions to test-common module (#17318)
This patch completely removes the compile-time dependency on core for both test and main sources by introducing two new modules.

1) `test-common` include all the common test implementation code (including dependency on :core for BrokerServer, ControllerServer, etc)
2) `test-common:api` new sub-module that just includes interfaces including our junit extension

Reviewers: David Arthur <mumrah@gmail.com>
2024-10-03 10:28:37 +08:00
Justine Olshan ae6e53fab2
MINOR: Fix MockAdminClient to match the server side update features handling. (#17343)
49d7ea6 updated the behavior of the UpdateFeaturesRequest/Response, but the MockAdminClient did not reflect those changes.

Now if any feature fails, all the features fail and the correct message is written in the result. Also only update the features if all features are successful and the command is not validate only.

Reviewers: Jun Rao <jun@confluent.io>
2024-10-02 13:20:44 -07:00
Justine Olshan 49d7ea6c6a
KAFKA-16308 [3/N]: Introduce feature dependency validation to UpdateFeatures command (#16443)
This change includes:

1. Dependency checking when updating the feature (all request versions)
2. Returning top level error and no feature level errors if any feature failed to update and using this error for all the features in the response. (all request versions)
3. Returning only top level none error for v2 and beyond

Reviewers: Jun Rao <jun@confluent.io>
2024-10-01 14:21:38 -07:00
Arnav Dadarya 05696037d3
KAFKA-12823 Remove Deprecated method KStream#through (#16761)
Implements KIP-1087

Reviewers: Matthias J. Sax <matthias@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-09-28 15:21:26 -07:00
PoAn Yang 10c789416c
KAFKA-17619: Remove zk type and instance from ClusterTest (#17284)
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
2024-09-27 11:38:15 -04:00
Ritika Reddy 6cb70a831c
KAFKA-17277: [2/2] Add feature dependency command to the storage and the feature command tool (#17093)
This patch belongs to the ongoing efforts of implementing KIP-1022.

Added feature-dependencies command to look up dependencies for a given feature version supplied by --feature flag. If the feature is not known or the version not yet defined, we throw an error.

Examples :
bin/kafka-feature feature-dependencies --feature transaction.version=2
transaction.version=2 requires:
metadata.version=4 (3.3-IV0) (listing any other version dependencies)

bin/kafka-feature feature-dependencies --feature metadata.version=17
metadata.version=17 (3.7-IV2) has no dependencies

Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits <alivshits@confluent.io>
2024-09-26 15:44:08 -07:00
PoAn Yang cd4d6ce9d5
MINOR: fix failed cases in FeatureCommandTest (#17287)
Reviewers: David Arthur <mumrah@gmail.com>
2024-09-26 11:24:09 -04:00
TengYao Chi 84fee3399e
KAFKA-17087 Deprecate `delete-config` of TopicCommand (#17222)
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-26 03:37:10 +08:00
Ken Huang 79753594ca
KAFKA-16813 Add global timeout (60s) for `@ClusterTemplate`, `@ClusterTest` and `@ClusterTests` (#16957)
Reviewers: TaiJuWu <tjwu1217@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-09-21 02:36:13 +08:00
Ken Huang 95b734d3c8
KAFKA-17472 Speed Up DescribeConsumerGroupTest (#17117)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-18 04:56:45 +08:00
Chung, Ming-Yen 60707a5fe9
KAFKA-17392 Remove whitelist option in ConsoleConsumerOptions (#17138)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-12 01:46:16 +08:00
TaiJuWu 74f0c9dabc
MINOR: a bit refacotr for TopicCommandTest (#17161)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-12 01:17:08 +08:00
Kuan-Po Tseng 089cbefac9
KAFKA-16830 Remove the scala version formatters support (#17127)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-10 16:04:58 +08:00
TaiJuWu 04dee3b2f2
KAFKA-17477 Migrate TopicCommand test to new test infra (#16127)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-09 02:21:23 +08:00
David Jacot 9abb8d3b3c
MINOR: Set `group.coordinator.rebalance.protocols` to `classic,consumer` by default (#17057)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-09-05 13:50:20 +08:00
Sasaki Toru 748d20200f
MINOR: Fix broken output layout of kafka-consumer-groups.sh (#17058)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Lianet Magrans <lmagrans@confluent.io>
2024-09-04 16:31:59 -04:00
Ritika Reddy edac19ba50
KAFKA-17277: [1/2] Add version mapping command to the storage tool and feature command tool (#16973)
As a part of KIP-1022 the following has been implemented in this patch:

A version-mapping command to to look up the corresponding features for a given metadata version. Using the command with no --release-version argument will return the mapping for the latest stable metadata version.
This command has been added to the FeatureCommand Tool and the Storage Tool.
The storage tools parsing method has been made more modular similar to the feature command tool

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-09-03 15:48:36 -07:00
Mickael Maison 1841c07d4a
KAFKA-17449 Move Quota classes to server-common module (#17060)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-31 12:41:34 +08:00
PoAn Yang 4a3ab89f95
KAFKA-17386 Remove broker-list, threads and num-fetch-threads in ConsumerPerformance (#16983)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-30 22:09:37 +08:00
TaiJuWu 165076afc6
KAFKA-17390 Remove broker-list in GetOffsetShell (#16992)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-30 00:16:21 +08:00
Logan Zhu 464051929d
KAFKA-17388 Remove broker-list from VerifiableProducer (#16958) 2024-08-29 20:02:29 +08:00
David Jacot c977bfdd3c
KAFKA-17413; Re-introduce `group.version` feature flag (#17013)
This patch re-introduces the `group.version` feature flag and gates the new consumer rebalance protocol with it. The `group.version` feature flag is attached to the metadata version `4.0-IV0` and it is marked as production ready. This allows system tests to pick it up directly by default without requiring to set `unstable.feature.versions.enable` in all of them. This is fine because we don't plan to do any incompatible changes before 4.0.

Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-29 01:22:54 -07:00
xijiu cb19a557e9
KAFKA-17373 Add print.epoch to kafka-console-share-consumer.sh/kafka-console-consumer.sh (#16987)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-26 17:30:57 +08:00
David Jacot aaf887d3d9
KAFKA-14048; [2/2] Use the new group coordinator by default in 4.0 (#16945)
This patch makes the new group coordinator, introduced as part of KIP-848, the default. This means that any KRaft cluster created from trunk defaults to using the new group coordinator. This includes all the integration tests which do not specify it. This patch also changes the default in system tests.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-26 01:14:26 -07:00
Sasaki Toru e37f1dfb25
MINOR: Fix an incorrect message in kafka-consumer-groups.sh when missing necessary options (#16961)
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-25 20:44:48 +08:00
Caio Guedes c7f0ade7e0
KAFKA-16334: Remove --bootstrap-servers from kafka-streams-application-reset
Reviewers: Matthias J. Sax <matthias@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-08-22 18:55:25 -07:00
PoAn Yang 28cd7136fb
KAFKA-16324 Move BrokerApiVersionsCommand to tools (#16850)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-22 09:22:20 +08:00
Andrew Schofield f6bfa9406c
KAFKA-17350: Improve share group describe for empty groups (#16897)
When you use kafka-share-groups.sh --describe for an empty group, it prints an empty table consisting of only the table header. kafka-consumer-groups.sh summarises the group status to make the output more informative and only prints the table if it contains more than zero rows.

This PR applies this principle across all of the variants of describing share groups which makes the output much nicer where the output would otherwise be strangely empty.

Reviewers:  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-08-20 12:05:08 +05:30
Kuan-Po Tseng 4271565409
KAFKA-16900 kafka-producer-perf-test reports error when using transaction. (#16646)
Currently, users need to set --transaction-duration-ms to enable transactions in kafka-producer-perf-test, which is not straightforward. A better approach is to enable transactions when a transaction ID is provided.

This PR allows enabling transaction in kafka-producer-perf-test by either

- set transaction.id=<id> via --producer-props or
- set transaction.id=<id> in config file via --producer.config or
- set --transaction-id <id> or
- set --transaction-duration-ms=<ms>

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-20 10:23:55 +08:00
Andrew Schofield 34475070e1
KAFKA-17368 Add delivery count to kafka-console-share-consumer.sh (#16925)
Now that ConsumerRecord.deliveryCount() exists, enhance kafka-console-share-consumer.sh to exploit it. Added support to the DefaultMessageFormatter and the option print.delivery to the usage message for kafka-console-share-consumer.sh. Note that it was not added to kafka-console-consumer.sh even though the option would be recognised - this is because delivery with a consumer group does not count deliveries, and the result would include Delivery:NOT_PRESENT for all records if it was enabled - not really that useful with a consumer group.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-20 06:19:36 +08:00
ShivsundarR 8cfd631264
KAFKA-16723 : Added kafka-console-share-consumer.sh tool. (#16860)
Added kafka-console-share-consumer.sh which will start a share consumer on a share group.
This tool helps to read data from Kafka topics using share groups and outputs it to standard output.

Reviewers:  Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>,  Manikumar Reddy <manikumar.reddy@gmail.com>
2024-08-16 15:26:54 +05:30
Andrew Schofield d64f4b9cd0
KAFKA-16714: kafka-share-groups.sh list and describe (#16835)
Introduces kafka-share-groups.sh for listing and describing share groups. The PR also contains the remaining options in the command parser and usage message in preparation of their implementation.

Reviewers:  Manikumar Reddy <manikumar.reddy@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
2024-08-14 16:22:26 +05:30