KAFKA-18291 Remove deprecated methods of ListConsumerGroupOffsetsOptions (#18265)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Nick Guo 2024-12-24 01:37:10 +08:00 committed by GitHub
parent 6fc5c64928
commit 1cf514313e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
4 changed files with 11 additions and 51 deletions

View File

@ -903,9 +903,7 @@ public interface Admin extends AutoCloseable {
* @return The ListConsumerGroupOffsetsResult
*/
default ListConsumerGroupOffsetsResult listConsumerGroupOffsets(String groupId, ListConsumerGroupOffsetsOptions options) {
@SuppressWarnings("deprecation")
ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec()
.topicPartitions(options.topicPartitions());
ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec();
// We can use the provided options with the batched API, which uses topic partitions from
// the group spec and ignores any topic partitions set in the options.

View File

@ -17,10 +17,8 @@
package org.apache.kafka.clients.admin;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.annotation.InterfaceStability;
import java.util.List;
/**
* Options for {@link Admin#listConsumerGroupOffsets(java.util.Map)} and {@link Admin#listConsumerGroupOffsets(String)}.
@ -30,26 +28,8 @@ import java.util.List;
@InterfaceStability.Evolving
public class ListConsumerGroupOffsetsOptions extends AbstractOptions<ListConsumerGroupOffsetsOptions> {
private List<TopicPartition> topicPartitions;
private boolean requireStable = false;
/**
* Set the topic partitions to list as part of the result.
* {@code null} includes all topic partitions.
* <p>
* @deprecated Since 3.3.
* Use {@link Admin#listConsumerGroupOffsets(java.util.Map, ListConsumerGroupOffsetsOptions)}
* to specify topic partitions.
*
* @param topicPartitions List of topic partitions to include
* @return This ListGroupOffsetsOptions
*/
@Deprecated
public ListConsumerGroupOffsetsOptions topicPartitions(List<TopicPartition> topicPartitions) {
this.topicPartitions = topicPartitions;
return this;
}
/**
* Sets an optional requireStable flag.
*/
@ -58,18 +38,6 @@ public class ListConsumerGroupOffsetsOptions extends AbstractOptions<ListConsume
return this;
}
/**
* Returns a list of topic partitions to add as part of the result.
* <p>
* @deprecated Since 3.3.
* Use {@link Admin#listConsumerGroupOffsets(java.util.Map, ListConsumerGroupOffsetsOptions)}
* to specify topic partitions.
*/
@Deprecated
public List<TopicPartition> topicPartitions() {
return topicPartitions;
}
public boolean requireStable() {
return requireStable;
}

View File

@ -4171,18 +4171,12 @@ public class KafkaAdminClientTest {
}
}
@Test
public void testListConsumerGroupOffsetsOptionsWithUnbatchedApi() throws Exception {
verifyListConsumerGroupOffsetsOptions(false);
}
@Test
public void testListConsumerGroupOffsetsOptionsWithBatchedApi() throws Exception {
verifyListConsumerGroupOffsetsOptions(true);
verifyListConsumerGroupOffsetsOptions();
}
@SuppressWarnings("deprecation")
private void verifyListConsumerGroupOffsetsOptions(boolean batchedApi) throws Exception {
private void verifyListConsumerGroupOffsetsOptions() throws Exception {
final Cluster cluster = mockCluster(3, 0);
final Time time = new MockTime();
@ -4196,13 +4190,10 @@ public class KafkaAdminClientTest {
final ListConsumerGroupOffsetsOptions options = new ListConsumerGroupOffsetsOptions()
.requireStable(true)
.timeoutMs(300);
if (batchedApi) {
final ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec()
.topicPartitions(partitions);
env.adminClient().listConsumerGroupOffsets(Collections.singletonMap(GROUP_ID, groupSpec), options);
} else {
env.adminClient().listConsumerGroupOffsets(GROUP_ID, options.topicPartitions(partitions));
}
final ListConsumerGroupOffsetsSpec groupSpec = new ListConsumerGroupOffsetsSpec()
.topicPartitions(partitions);
env.adminClient().listConsumerGroupOffsets(Collections.singletonMap(GROUP_ID, groupSpec), options);
final MockClient mockClient = env.kafkaClient();
waitForRequest(mockClient, ApiKeys.OFFSET_FETCH);

View File

@ -211,9 +211,12 @@
<code>DEAD</code> if the group ID was not found. In Apache Kafka 4.0, the <code>GroupIdNotFoundException</code>
is thrown instead as part of the support for new types of group.
</li>
<li>The <code>org.apache.kafka.clients.admin.TopicListing.TopicListing(String, boolean)</code> method were removed.
<li>The <code>org.apache.kafka.clients.admin.TopicListing.TopicListing(String, boolean)</code> method was removed.
Please use <code>org.apache.kafka.clients.admin.TopicListing.TopicListing(String, Uuid, boolean)</code> instead.
</li>
<li>The <code>org.apache.kafka.clients.admin.ListConsumerGroupOffsetsOptions.topicPartitions(List&lt;TopicPartition&gt;)</code> method was removed.
Please use <code>org.apache.kafka.clients.admin.Admin.listConsumerGroupOffsets(Map&lt;String, ListConsumerGroupOffsetsSpec&gt;, ListConsumerGroupOffsetsOptions)</code> instead.
</li>
<li>
The deprecated <code>dryRun</code> methods were removed from the <code>org.apache.kafka.clients.admin.UpdateFeaturesOptions</code>.
Please use <code>validateOnly</code> instead.