Skip to content

KAFKA-19500: kafka-consumer-groups.sh should fail quickly if the partition leader is unavailable #20168

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 6 commits into from
Jul 21, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,8 @@
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.GroupIdNotFoundException;
import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.server.util.CommandLineUtils;
Expand Down Expand Up @@ -1000,6 +1002,9 @@ private Map<TopicPartition, OffsetAndMetadata> getCommittedOffsets(String groupI
}

private Map<TopicPartition, OffsetAndMetadata> prepareOffsetsToReset(String groupId, Collection<TopicPartition> partitionsToReset) {
// ensure all partitions are valid, otherwise throw a runtime exception
checkAllTopicPartitionsValid(partitionsToReset);

if (opts.options.has(opts.resetToOffsetOpt)) {
return offsetsUtils.resetToOffset(partitionsToReset);
} else if (opts.options.has(opts.resetToEarliestOpt)) {
Expand All @@ -1024,6 +1029,38 @@ private Map<TopicPartition, OffsetAndMetadata> prepareOffsetsToReset(String grou
return null;
}

private void checkAllTopicPartitionsValid(Collection<TopicPartition> partitionsToReset) {
// check the partitions exist
List<TopicPartition> partitionsNotExistList = filterNonExistentPartitions(partitionsToReset);
if (!partitionsNotExistList.isEmpty()) {
String partitionStr = partitionsNotExistList.stream().map(TopicPartition::toString).collect(Collectors.joining(","));
throw new UnknownTopicOrPartitionException("The partitions \"" + partitionStr + "\" do not exist");
}

// check the partitions have leader
List<TopicPartition> partitionsWithoutLeader = filterNoneLeaderPartitions(partitionsToReset);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if there is a topic having three partitions, and the t-2 partition is offline, then if partitionsToReset is t-0,t-1, filterNoneLeaderPartitions will return t-2, causing the tool to fail. Is it expected?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it's fair enough.

if (!partitionsWithoutLeader.isEmpty()) {
String partitionStr = partitionsWithoutLeader.stream().map(TopicPartition::toString).collect(Collectors.joining(","));
throw new LeaderNotAvailableException("The partitions \"" + partitionStr + "\" have no leader");
}
}

private List<TopicPartition> filterNonExistentPartitions(Collection<TopicPartition> topicPartitions) {
// collect all topics
Set<String> topics = topicPartitions.stream().map(TopicPartition::topic).collect(Collectors.toSet());
try {
List<TopicPartition> existPartitions = adminClient.describeTopics(topics).allTopicNames().get().entrySet()
.stream()
.flatMap(entry -> entry.getValue().partitions().stream()
.map(partitionInfo -> new TopicPartition(entry.getKey(), partitionInfo.partition())))
.toList();

return topicPartitions.stream().filter(element -> !existPartitions.contains(element)).toList();
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
}

String exportOffsetsToCsv(Map<String, Map<TopicPartition, OffsetAndMetadata>> assignments) {
boolean isSingleGroupQuery = opts.options.valuesOf(opts.groupOpt).size() == 1;
ObjectWriter csvWriter = isSingleGroupQuery
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anySet;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
Expand Down Expand Up @@ -234,6 +235,8 @@ public void testAdminRequestsForResetOffsets() {
.thenReturn(describeGroupsResult(GroupState.DEAD));
when(admin.describeTopics(ArgumentMatchers.eq(topicsWithoutPartitionsSpecified), any()))
.thenReturn(describeTopicsResult(topicsWithoutPartitionsSpecified));
when(admin.describeTopics(anySet()))
.thenReturn(describeTopicsResult(TOPICS));
when(admin.listOffsets(offsetsArgMatcher(), any()))
.thenReturn(listOffsetsResult());

Expand Down Expand Up @@ -317,7 +320,7 @@ private DescribeTopicsResult describeTopicsResult(Collection<String> topics) {

topics.forEach(topic -> {
List<TopicPartitionInfo> partitions = IntStream.range(0, NUM_PARTITIONS)
.mapToObj(i -> new TopicPartitionInfo(i, null, Collections.emptyList(), Collections.emptyList()))
.mapToObj(i -> new TopicPartitionInfo(i, Node.noNode(), Collections.emptyList(), Collections.emptyList()))
.collect(Collectors.toList());
topicDescriptions.put(topic, new TopicDescription(topic, false, partitions));
});
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.GroupState;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.LeaderNotAvailableException;
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.test.ClusterInstance;
Expand Down Expand Up @@ -81,6 +83,7 @@
import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG;
import static org.apache.kafka.coordinator.group.GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG;
import static org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
Expand Down Expand Up @@ -659,6 +662,41 @@ public void testResetWithUnrecognizedNewConsumerOption(ClusterInstance cluster)
assertThrows(OptionException.class, () -> getConsumerGroupService(cgcArgs));
}

@ClusterTest(brokers = 3, serverProperties = {@ClusterConfigProperty(key = OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "2")})
public void testResetOffsetsWithPartitionNoneLeader(ClusterInstance cluster) throws Exception {
String group = generateRandomGroupId();
String topic = generateRandomTopic();
String[] args = buildArgsForGroup(cluster, group, "--topic", topic + ":0,1,2",
"--to-earliest", "--execute");

try (Admin admin = cluster.admin();
ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) {

admin.createTopics(singleton(new NewTopic(topic, 3, (short) 1))).all().get();
produceConsumeAndShutdown(cluster, topic, group, 2, GroupProtocol.CLASSIC);
assertDoesNotThrow(() -> resetOffsets(service));
// shutdown a broker to make some partitions missing leader
cluster.shutdownBroker(0);
assertThrows(LeaderNotAvailableException.class, () -> resetOffsets(service));
}
}

@ClusterTest
public void testResetOffsetsWithPartitionNotExist(ClusterInstance cluster) throws Exception {
String group = generateRandomGroupId();
String topic = generateRandomTopic();
String[] args = buildArgsForGroup(cluster, group, "--topic", topic + ":2,3",
"--to-earliest", "--execute");

try (Admin admin = cluster.admin();
ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) {

admin.createTopics(singleton(new NewTopic(topic, 1, (short) 1))).all().get();
produceConsumeAndShutdown(cluster, topic, group, 2, GroupProtocol.CLASSIC);
assertThrows(UnknownTopicOrPartitionException.class, () -> resetOffsets(service));
}
}

private String generateRandomTopic() {
return TOPIC_PREFIX + TestUtils.randomString(10);
}
Expand Down