mirror of https://github.com/apache/kafka.git
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>
This commit is contained in:
parent
8864cba0e8
commit
e7d986e48c
|
@ -64,6 +64,7 @@ public class DescribeConsumerGroupsHandler implements AdminApiHandler<Coordinato
|
||||||
private final Logger log;
|
private final Logger log;
|
||||||
private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
|
private final AdminApiLookupStrategy<CoordinatorKey> lookupStrategy;
|
||||||
private final Set<String> useClassicGroupApi;
|
private final Set<String> useClassicGroupApi;
|
||||||
|
private final Map<String, String> groupIdNotFoundErrorMessages;
|
||||||
|
|
||||||
public DescribeConsumerGroupsHandler(
|
public DescribeConsumerGroupsHandler(
|
||||||
boolean includeAuthorizedOperations,
|
boolean includeAuthorizedOperations,
|
||||||
|
@ -73,6 +74,7 @@ public class DescribeConsumerGroupsHandler implements AdminApiHandler<Coordinato
|
||||||
this.log = logContext.logger(DescribeConsumerGroupsHandler.class);
|
this.log = logContext.logger(DescribeConsumerGroupsHandler.class);
|
||||||
this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext);
|
this.lookupStrategy = new CoordinatorStrategy(CoordinatorType.GROUP, logContext);
|
||||||
this.useClassicGroupApi = new HashSet<>();
|
this.useClassicGroupApi = new HashSet<>();
|
||||||
|
this.groupIdNotFoundErrorMessages = new HashMap<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static Set<CoordinatorKey> buildKeySet(Collection<String> groupIds) {
|
private static Set<CoordinatorKey> buildKeySet(Collection<String> groupIds) {
|
||||||
|
@ -255,7 +257,7 @@ public class DescribeConsumerGroupsHandler implements AdminApiHandler<Coordinato
|
||||||
handleError(
|
handleError(
|
||||||
groupIdKey,
|
groupIdKey,
|
||||||
error,
|
error,
|
||||||
null,
|
describedGroup.errorMessage(),
|
||||||
failed,
|
failed,
|
||||||
groupsToUnmap,
|
groupsToUnmap,
|
||||||
false
|
false
|
||||||
|
@ -354,11 +356,18 @@ public class DescribeConsumerGroupsHandler implements AdminApiHandler<Coordinato
|
||||||
case GROUP_ID_NOT_FOUND:
|
case GROUP_ID_NOT_FOUND:
|
||||||
if (isConsumerGroupResponse) {
|
if (isConsumerGroupResponse) {
|
||||||
log.debug("`{}` request for group id {} failed because the group is not " +
|
log.debug("`{}` request for group id {} failed because the group is not " +
|
||||||
"a new consumer group. Will retry with `DescribeGroups` API.", apiName, groupId.idValue);
|
"a new consumer group. Will retry with `DescribeGroups` API. {}",
|
||||||
|
apiName, groupId.idValue, errorMsg != null ? errorMsg : "");
|
||||||
useClassicGroupApi.add(groupId.idValue);
|
useClassicGroupApi.add(groupId.idValue);
|
||||||
|
|
||||||
|
// The error message from the ConsumerGroupDescribe API is more informative to the user
|
||||||
|
// than the error message from the classic group API. Capture it and use it if we get the
|
||||||
|
// same error code for the classic group API also.
|
||||||
|
groupIdNotFoundErrorMessages.put(groupId.idValue, errorMsg);
|
||||||
} else {
|
} else {
|
||||||
log.error("`{}` request for group id {} failed because the group does not exist.", apiName, groupId.idValue);
|
log.debug("`{}` request for group id {} failed because the group does not exist. {}",
|
||||||
failed.put(groupId, error.exception(errorMsg));
|
apiName, groupId.idValue, errorMsg != null ? errorMsg : "");
|
||||||
|
failed.put(groupId, error.exception(groupIdNotFoundErrorMessages.getOrDefault(groupId.idValue, errorMsg)));
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
|
|
||||||
|
|
|
@ -37,7 +37,6 @@ import org.slf4j.Logger;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -177,17 +176,9 @@ public class DescribeShareGroupsHandler extends AdminApiHandler.Batched<Coordina
|
||||||
break;
|
break;
|
||||||
|
|
||||||
case GROUP_ID_NOT_FOUND:
|
case GROUP_ID_NOT_FOUND:
|
||||||
// In order to maintain compatibility with describeConsumerGroups, an unknown group ID is
|
|
||||||
// reported as a DEAD share group, and the admin client operation did not fail
|
|
||||||
log.debug("`DescribeShareGroups` request for group id {} failed because the group does not exist. {}",
|
log.debug("`DescribeShareGroups` request for group id {} failed because the group does not exist. {}",
|
||||||
groupId.idValue, errorMsg != null ? errorMsg : "");
|
groupId.idValue, errorMsg != null ? errorMsg : "");
|
||||||
final ShareGroupDescription shareGroupDescription =
|
failed.put(groupId, error.exception(errorMsg));
|
||||||
new ShareGroupDescription(groupId.idValue,
|
|
||||||
Collections.emptySet(),
|
|
||||||
GroupState.DEAD,
|
|
||||||
coordinator,
|
|
||||||
validAclOperations(describedGroup.authorizedOperations()));
|
|
||||||
completed.put(groupId, shareGroupDescription);
|
|
||||||
break;
|
break;
|
||||||
|
|
||||||
default:
|
default:
|
||||||
|
|
|
@ -110,6 +110,14 @@ public class DescribeGroupsResponse extends AbstractResponse {
|
||||||
DescribeGroupsResponse.UNKNOWN_PROTOCOL, Collections.emptyList(), AUTHORIZED_OPERATIONS_OMITTED);
|
DescribeGroupsResponse.UNKNOWN_PROTOCOL, Collections.emptyList(), AUTHORIZED_OPERATIONS_OMITTED);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static DescribedGroup groupError(String groupId, Errors error, String errorMessage) {
|
||||||
|
return new DescribedGroup()
|
||||||
|
.setGroupId(groupId)
|
||||||
|
.setGroupState(DescribeGroupsResponse.UNKNOWN_STATE)
|
||||||
|
.setErrorCode(error.code())
|
||||||
|
.setErrorMessage(errorMessage);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DescribeGroupsResponseData data() {
|
public DescribeGroupsResponseData data() {
|
||||||
return data;
|
return data;
|
||||||
|
|
|
@ -25,11 +25,13 @@
|
||||||
// Starting in version 4, the response will include group.instance.id info for members.
|
// Starting in version 4, the response will include group.instance.id info for members.
|
||||||
//
|
//
|
||||||
// Version 5 is the first flexible version.
|
// Version 5 is the first flexible version.
|
||||||
"validVersions": "0-5",
|
//
|
||||||
|
// Version 6 returns error code GROUP_ID_NOT_FOUND if the group ID is not found (KIP-1043).
|
||||||
|
"validVersions": "0-6",
|
||||||
"flexibleVersions": "5+",
|
"flexibleVersions": "5+",
|
||||||
"fields": [
|
"fields": [
|
||||||
{ "name": "Groups", "type": "[]string", "versions": "0+", "entityType": "groupId",
|
{ "name": "Groups", "type": "[]string", "versions": "0+", "entityType": "groupId",
|
||||||
"about": "The names of the groups to describe" },
|
"about": "The names of the groups to describe." },
|
||||||
{ "name": "IncludeAuthorizedOperations", "type": "bool", "versions": "3+",
|
{ "name": "IncludeAuthorizedOperations", "type": "bool", "versions": "3+",
|
||||||
"about": "Whether to include authorized operations." }
|
"about": "Whether to include authorized operations." }
|
||||||
]
|
]
|
||||||
|
|
|
@ -26,7 +26,9 @@
|
||||||
// Starting in version 4, the response will optionally include group.instance.id info for members.
|
// Starting in version 4, the response will optionally include group.instance.id info for members.
|
||||||
//
|
//
|
||||||
// Version 5 is the first flexible version.
|
// Version 5 is the first flexible version.
|
||||||
"validVersions": "0-5",
|
//
|
||||||
|
// Version 6 returns error code GROUP_ID_NOT_FOUND if the group ID is not found (KIP-1043).
|
||||||
|
"validVersions": "0-6",
|
||||||
"flexibleVersions": "5+",
|
"flexibleVersions": "5+",
|
||||||
"fields": [
|
"fields": [
|
||||||
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
|
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true,
|
||||||
|
@ -35,6 +37,8 @@
|
||||||
"about": "Each described group.", "fields": [
|
"about": "Each described group.", "fields": [
|
||||||
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
{ "name": "ErrorCode", "type": "int16", "versions": "0+",
|
||||||
"about": "The describe error, or 0 if there was no error." },
|
"about": "The describe error, or 0 if there was no error." },
|
||||||
|
{ "name": "ErrorMessage", "type": "string", "versions": "6+", "nullableVersions": "6+", "default": "null",
|
||||||
|
"about": "The describe error message, or null if there was no error." },
|
||||||
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
|
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
|
||||||
"about": "The group ID string." },
|
"about": "The group ID string." },
|
||||||
{ "name": "GroupState", "type": "string", "versions": "0+",
|
{ "name": "GroupState", "type": "string", "versions": "0+",
|
||||||
|
|
|
@ -3799,7 +3799,13 @@ public class KafkaAdminClientTest {
|
||||||
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) {
|
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) {
|
||||||
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
|
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
|
||||||
|
|
||||||
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
|
env.kafkaClient().prepareResponse(new FindCoordinatorResponse(
|
||||||
|
new FindCoordinatorResponseData()
|
||||||
|
.setCoordinators(asList(
|
||||||
|
FindCoordinatorResponse.prepareCoordinatorResponse(Errors.NONE, GROUP_ID, env.cluster().controller()),
|
||||||
|
FindCoordinatorResponse.prepareCoordinatorResponse(Errors.NONE, "group-connect-0", env.cluster().controller())
|
||||||
|
))
|
||||||
|
));
|
||||||
|
|
||||||
// The first request sent will be a ConsumerGroupDescribe request. Let's
|
// The first request sent will be a ConsumerGroupDescribe request. Let's
|
||||||
// fail it in order to fail back to using the classic version.
|
// fail it in order to fail back to using the classic version.
|
||||||
|
@ -3819,8 +3825,8 @@ public class KafkaAdminClientTest {
|
||||||
byte[] memberAssignmentBytes = new byte[memberAssignment.remaining()];
|
byte[] memberAssignmentBytes = new byte[memberAssignment.remaining()];
|
||||||
memberAssignment.get(memberAssignmentBytes);
|
memberAssignment.get(memberAssignmentBytes);
|
||||||
|
|
||||||
DescribeGroupsResponseData group0Data = new DescribeGroupsResponseData();
|
DescribeGroupsResponseData groupData = new DescribeGroupsResponseData();
|
||||||
group0Data.groups().add(DescribeGroupsResponse.groupMetadata(
|
groupData.groups().add(DescribeGroupsResponse.groupMetadata(
|
||||||
GROUP_ID,
|
GROUP_ID,
|
||||||
Errors.NONE,
|
Errors.NONE,
|
||||||
"",
|
"",
|
||||||
|
@ -3831,9 +3837,7 @@ public class KafkaAdminClientTest {
|
||||||
DescribeGroupsResponse.groupMember("1", null, "clientId1", "clientHost", memberAssignmentBytes, null)
|
DescribeGroupsResponse.groupMember("1", null, "clientId1", "clientHost", memberAssignmentBytes, null)
|
||||||
),
|
),
|
||||||
Collections.emptySet()));
|
Collections.emptySet()));
|
||||||
|
groupData.groups().add(DescribeGroupsResponse.groupMetadata(
|
||||||
DescribeGroupsResponseData groupConnectData = new DescribeGroupsResponseData();
|
|
||||||
group0Data.groups().add(DescribeGroupsResponse.groupMetadata(
|
|
||||||
"group-connect-0",
|
"group-connect-0",
|
||||||
Errors.NONE,
|
Errors.NONE,
|
||||||
"",
|
"",
|
||||||
|
@ -3845,8 +3849,7 @@ public class KafkaAdminClientTest {
|
||||||
),
|
),
|
||||||
Collections.emptySet()));
|
Collections.emptySet()));
|
||||||
|
|
||||||
env.kafkaClient().prepareResponse(new DescribeGroupsResponse(group0Data));
|
env.kafkaClient().prepareResponse(new DescribeGroupsResponse(groupData));
|
||||||
env.kafkaClient().prepareResponse(new DescribeGroupsResponse(groupConnectData));
|
|
||||||
|
|
||||||
Collection<String> groups = new HashSet<>();
|
Collection<String> groups = new HashSet<>();
|
||||||
groups.add(GROUP_ID);
|
groups.add(GROUP_ID);
|
||||||
|
@ -3854,6 +3857,72 @@ public class KafkaAdminClientTest {
|
||||||
final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(groups);
|
final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(groups);
|
||||||
assertEquals(2, result.describedGroups().size());
|
assertEquals(2, result.describedGroups().size());
|
||||||
assertEquals(groups, result.describedGroups().keySet());
|
assertEquals(groups, result.describedGroups().keySet());
|
||||||
|
KafkaFuture<Map<String, ConsumerGroupDescription>> allFuture = result.all();
|
||||||
|
// This throws because the second group is a classic connect group, not a consumer group.
|
||||||
|
assertThrows(ExecutionException.class, allFuture::get);
|
||||||
|
assertTrue(allFuture.isCompletedExceptionally());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDescribeConsumerGroupsGroupIdNotFound() {
|
||||||
|
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) {
|
||||||
|
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
|
||||||
|
|
||||||
|
env.kafkaClient().prepareResponse(new FindCoordinatorResponse(
|
||||||
|
new FindCoordinatorResponseData()
|
||||||
|
.setCoordinators(asList(
|
||||||
|
FindCoordinatorResponse.prepareCoordinatorResponse(Errors.NONE, GROUP_ID, env.cluster().controller()),
|
||||||
|
FindCoordinatorResponse.prepareCoordinatorResponse(Errors.NONE, "group-connect-0", env.cluster().controller())
|
||||||
|
))
|
||||||
|
));
|
||||||
|
|
||||||
|
// The first request sent will be a ConsumerGroupDescribe request. Let's
|
||||||
|
// fail it in order to fail back to using the classic version.
|
||||||
|
env.kafkaClient().prepareUnsupportedVersionResponse(
|
||||||
|
request -> request instanceof ConsumerGroupDescribeRequest);
|
||||||
|
|
||||||
|
TopicPartition myTopicPartition0 = new TopicPartition("my_topic", 0);
|
||||||
|
TopicPartition myTopicPartition1 = new TopicPartition("my_topic", 1);
|
||||||
|
TopicPartition myTopicPartition2 = new TopicPartition("my_topic", 2);
|
||||||
|
|
||||||
|
final List<TopicPartition> topicPartitions = new ArrayList<>();
|
||||||
|
topicPartitions.add(0, myTopicPartition0);
|
||||||
|
topicPartitions.add(1, myTopicPartition1);
|
||||||
|
topicPartitions.add(2, myTopicPartition2);
|
||||||
|
|
||||||
|
final ByteBuffer memberAssignment = ConsumerProtocol.serializeAssignment(new ConsumerPartitionAssignor.Assignment(topicPartitions));
|
||||||
|
byte[] memberAssignmentBytes = new byte[memberAssignment.remaining()];
|
||||||
|
memberAssignment.get(memberAssignmentBytes);
|
||||||
|
|
||||||
|
DescribeGroupsResponseData groupData = new DescribeGroupsResponseData();
|
||||||
|
groupData.groups().add(DescribeGroupsResponse.groupMetadata(
|
||||||
|
GROUP_ID,
|
||||||
|
Errors.NONE,
|
||||||
|
"",
|
||||||
|
ConsumerProtocol.PROTOCOL_TYPE,
|
||||||
|
"",
|
||||||
|
asList(
|
||||||
|
DescribeGroupsResponse.groupMember("0", null, "clientId0", "clientHost", memberAssignmentBytes, null),
|
||||||
|
DescribeGroupsResponse.groupMember("1", null, "clientId1", "clientHost", memberAssignmentBytes, null)
|
||||||
|
),
|
||||||
|
Collections.emptySet()));
|
||||||
|
groupData.groups().add(DescribeGroupsResponse.groupError(
|
||||||
|
"group-connect-0",
|
||||||
|
Errors.GROUP_ID_NOT_FOUND,
|
||||||
|
"Group group-connect-0 is not a classic group."));
|
||||||
|
|
||||||
|
env.kafkaClient().prepareResponse(new DescribeGroupsResponse(groupData));
|
||||||
|
|
||||||
|
Collection<String> groups = new HashSet<>();
|
||||||
|
groups.add(GROUP_ID);
|
||||||
|
groups.add("group-connect-0");
|
||||||
|
final DescribeConsumerGroupsResult result = env.adminClient().describeConsumerGroups(groups);
|
||||||
|
assertEquals(2, result.describedGroups().size());
|
||||||
|
assertEquals(groups, result.describedGroups().keySet());
|
||||||
|
KafkaFuture<Map<String, ConsumerGroupDescription>> allFuture = result.all();
|
||||||
|
assertThrows(ExecutionException.class, allFuture::get);
|
||||||
|
assertTrue(result.all().isCompletedExceptionally());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -4996,6 +5065,59 @@ public class KafkaAdminClientTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDescribeShareGroupsGroupIdNotFound() throws Exception {
|
||||||
|
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) {
|
||||||
|
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
|
||||||
|
|
||||||
|
env.kafkaClient().prepareResponse(new FindCoordinatorResponse(
|
||||||
|
new FindCoordinatorResponseData()
|
||||||
|
.setCoordinators(asList(
|
||||||
|
FindCoordinatorResponse.prepareCoordinatorResponse(Errors.NONE, GROUP_ID, env.cluster().controller()),
|
||||||
|
FindCoordinatorResponse.prepareCoordinatorResponse(Errors.NONE, "group-1", env.cluster().controller())
|
||||||
|
))
|
||||||
|
));
|
||||||
|
|
||||||
|
ShareGroupDescribeResponseData.TopicPartitions topicPartitions = new ShareGroupDescribeResponseData.TopicPartitions()
|
||||||
|
.setTopicName("my_topic")
|
||||||
|
.setPartitions(asList(0, 1, 2));
|
||||||
|
final ShareGroupDescribeResponseData.Assignment memberAssignment = new ShareGroupDescribeResponseData.Assignment()
|
||||||
|
.setTopicPartitions(asList(topicPartitions));
|
||||||
|
ShareGroupDescribeResponseData groupData = new ShareGroupDescribeResponseData();
|
||||||
|
groupData.groups().add(new ShareGroupDescribeResponseData.DescribedGroup()
|
||||||
|
.setGroupId(GROUP_ID)
|
||||||
|
.setGroupState(GroupState.STABLE.toString())
|
||||||
|
.setMembers(asList(
|
||||||
|
new ShareGroupDescribeResponseData.Member()
|
||||||
|
.setMemberId("0")
|
||||||
|
.setClientId("clientId0")
|
||||||
|
.setClientHost("clientHost")
|
||||||
|
.setAssignment(memberAssignment),
|
||||||
|
new ShareGroupDescribeResponseData.Member()
|
||||||
|
.setMemberId("1")
|
||||||
|
.setClientId("clientId1")
|
||||||
|
.setClientHost("clientHost")
|
||||||
|
.setAssignment(memberAssignment))));
|
||||||
|
groupData.groups().add(new ShareGroupDescribeResponseData.DescribedGroup()
|
||||||
|
.setGroupId("group-1")
|
||||||
|
.setGroupState(GroupState.DEAD.toString())
|
||||||
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
||||||
|
.setErrorMessage("Group group-1 not found."));
|
||||||
|
|
||||||
|
env.kafkaClient().prepareResponse(new ShareGroupDescribeResponse(groupData));
|
||||||
|
|
||||||
|
Collection<String> groups = new HashSet<>();
|
||||||
|
groups.add(GROUP_ID);
|
||||||
|
groups.add("group-1");
|
||||||
|
final DescribeShareGroupsResult result = env.adminClient().describeShareGroups(groups);
|
||||||
|
assertEquals(2, result.describedGroups().size());
|
||||||
|
assertEquals(groups, result.describedGroups().keySet());
|
||||||
|
KafkaFuture<Map<String, ShareGroupDescription>> allFuture = result.all();
|
||||||
|
assertThrows(ExecutionException.class, allFuture::get);
|
||||||
|
assertTrue(result.all().isCompletedExceptionally());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDescribeShareGroupsWithAuthorizedOperationsOmitted() throws Exception {
|
public void testDescribeShareGroupsWithAuthorizedOperationsOmitted() throws Exception {
|
||||||
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) {
|
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) {
|
||||||
|
@ -5024,15 +5146,21 @@ public class KafkaAdminClientTest {
|
||||||
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) {
|
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(mockCluster(1, 0))) {
|
||||||
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
|
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
|
||||||
|
|
||||||
env.kafkaClient().prepareResponse(prepareFindCoordinatorResponse(Errors.NONE, env.cluster().controller()));
|
env.kafkaClient().prepareResponse(new FindCoordinatorResponse(
|
||||||
|
new FindCoordinatorResponseData()
|
||||||
|
.setCoordinators(asList(
|
||||||
|
FindCoordinatorResponse.prepareCoordinatorResponse(Errors.NONE, GROUP_ID, env.cluster().controller()),
|
||||||
|
FindCoordinatorResponse.prepareCoordinatorResponse(Errors.NONE, "group-1", env.cluster().controller())
|
||||||
|
))
|
||||||
|
));
|
||||||
|
|
||||||
ShareGroupDescribeResponseData.TopicPartitions topicPartitions = new ShareGroupDescribeResponseData.TopicPartitions()
|
ShareGroupDescribeResponseData.TopicPartitions topicPartitions = new ShareGroupDescribeResponseData.TopicPartitions()
|
||||||
.setTopicName("my_topic")
|
.setTopicName("my_topic")
|
||||||
.setPartitions(asList(0, 1, 2));
|
.setPartitions(asList(0, 1, 2));
|
||||||
final ShareGroupDescribeResponseData.Assignment memberAssignment = new ShareGroupDescribeResponseData.Assignment()
|
final ShareGroupDescribeResponseData.Assignment memberAssignment = new ShareGroupDescribeResponseData.Assignment()
|
||||||
.setTopicPartitions(asList(topicPartitions));
|
.setTopicPartitions(asList(topicPartitions));
|
||||||
ShareGroupDescribeResponseData group0Data = new ShareGroupDescribeResponseData();
|
ShareGroupDescribeResponseData groupData = new ShareGroupDescribeResponseData();
|
||||||
group0Data.groups().add(new ShareGroupDescribeResponseData.DescribedGroup()
|
groupData.groups().add(new ShareGroupDescribeResponseData.DescribedGroup()
|
||||||
.setGroupId(GROUP_ID)
|
.setGroupId(GROUP_ID)
|
||||||
.setGroupState(GroupState.STABLE.toString())
|
.setGroupState(GroupState.STABLE.toString())
|
||||||
.setMembers(asList(
|
.setMembers(asList(
|
||||||
|
@ -5046,9 +5174,7 @@ public class KafkaAdminClientTest {
|
||||||
.setClientId("clientId1")
|
.setClientId("clientId1")
|
||||||
.setClientHost("clientHost")
|
.setClientHost("clientHost")
|
||||||
.setAssignment(memberAssignment))));
|
.setAssignment(memberAssignment))));
|
||||||
|
groupData.groups().add(new ShareGroupDescribeResponseData.DescribedGroup()
|
||||||
ShareGroupDescribeResponseData group1Data = new ShareGroupDescribeResponseData();
|
|
||||||
group1Data.groups().add(new ShareGroupDescribeResponseData.DescribedGroup()
|
|
||||||
.setGroupId("group-1")
|
.setGroupId("group-1")
|
||||||
.setGroupState(GroupState.STABLE.toString())
|
.setGroupState(GroupState.STABLE.toString())
|
||||||
.setMembers(asList(
|
.setMembers(asList(
|
||||||
|
@ -5063,8 +5189,7 @@ public class KafkaAdminClientTest {
|
||||||
.setClientHost("clientHost")
|
.setClientHost("clientHost")
|
||||||
.setAssignment(memberAssignment))));
|
.setAssignment(memberAssignment))));
|
||||||
|
|
||||||
env.kafkaClient().prepareResponse(new ShareGroupDescribeResponse(group0Data));
|
env.kafkaClient().prepareResponse(new ShareGroupDescribeResponse(groupData));
|
||||||
env.kafkaClient().prepareResponse(new ShareGroupDescribeResponse(group1Data));
|
|
||||||
|
|
||||||
Collection<String> groups = new HashSet<>();
|
Collection<String> groups = new HashSet<>();
|
||||||
groups.add(GROUP_ID);
|
groups.add(GROUP_ID);
|
||||||
|
@ -5072,6 +5197,9 @@ public class KafkaAdminClientTest {
|
||||||
final DescribeShareGroupsResult result = env.adminClient().describeShareGroups(groups);
|
final DescribeShareGroupsResult result = env.adminClient().describeShareGroups(groups);
|
||||||
assertEquals(2, result.describedGroups().size());
|
assertEquals(2, result.describedGroups().size());
|
||||||
assertEquals(groups, result.describedGroups().keySet());
|
assertEquals(groups, result.describedGroups().keySet());
|
||||||
|
KafkaFuture<Map<String, ShareGroupDescription>> allFuture = result.all();
|
||||||
|
assertDoesNotThrow(() -> allFuture.get());
|
||||||
|
assertFalse(allFuture.isCompletedExceptionally());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.kafka.clients.producer.RecordMetadata;
|
||||||
import org.apache.kafka.common.GroupState;
|
import org.apache.kafka.common.GroupState;
|
||||||
import org.apache.kafka.common.KafkaFuture;
|
import org.apache.kafka.common.KafkaFuture;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
||||||
import org.apache.kafka.common.errors.UnknownMemberIdException;
|
import org.apache.kafka.common.errors.UnknownMemberIdException;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
import org.apache.kafka.connect.data.Schema;
|
import org.apache.kafka.connect.data.Schema;
|
||||||
|
@ -301,6 +302,7 @@ public class MirrorCheckpointTask extends SourceTask {
|
||||||
// sync offset to the target cluster only if the state of current consumer group is:
|
// sync offset to the target cluster only if the state of current consumer group is:
|
||||||
// (1) idle: because the consumer at target is not actively consuming the mirrored topic
|
// (1) idle: because the consumer at target is not actively consuming the mirrored topic
|
||||||
// (2) dead: the new consumer that is recently created at source and never existed at target
|
// (2) dead: the new consumer that is recently created at source and never existed at target
|
||||||
|
// This case will be reported as a GroupIdNotFoundException
|
||||||
if (consumerGroupState == GroupState.EMPTY) {
|
if (consumerGroupState == GroupState.EMPTY) {
|
||||||
idleConsumerGroupsOffset.put(
|
idleConsumerGroupsOffset.put(
|
||||||
group,
|
group,
|
||||||
|
@ -311,8 +313,13 @@ public class MirrorCheckpointTask extends SourceTask {
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
// new consumer upstream has state "DEAD" and will be identified during the offset sync-up
|
// new consumer upstream has state "DEAD" and will be identified during the offset sync-up
|
||||||
} catch (InterruptedException | ExecutionException e) {
|
} catch (InterruptedException ie) {
|
||||||
log.error("Error querying for consumer group {} on cluster {}.", group, targetClusterAlias, e);
|
log.error("Error querying for consumer group {} on cluster {}.", group, targetClusterAlias, ie);
|
||||||
|
} catch (ExecutionException ee) {
|
||||||
|
// check for non-existent new consumer upstream which will be identified during the offset sync-up
|
||||||
|
if (!(ee.getCause() instanceof GroupIdNotFoundException)) {
|
||||||
|
log.error("Error querying for consumer group {} on cluster {}.", group, targetClusterAlias, ee);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1138,15 +1138,20 @@ private[group] class GroupCoordinator(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def handleDescribeGroup(groupId: String): (Errors, GroupSummary) = {
|
def handleDescribeGroup(groupId: String, apiVersion: Short): (Errors, Option[String], GroupSummary) = {
|
||||||
validateGroupStatus(groupId, ApiKeys.DESCRIBE_GROUPS) match {
|
validateGroupStatus(groupId, ApiKeys.DESCRIBE_GROUPS) match {
|
||||||
case Some(error) => (error, GroupCoordinator.EmptyGroup)
|
case Some(error) => (error, None, GroupCoordinator.EmptyGroup)
|
||||||
case None =>
|
case None =>
|
||||||
groupManager.getGroup(groupId) match {
|
groupManager.getGroup(groupId) match {
|
||||||
case None => (Errors.NONE, GroupCoordinator.DeadGroup)
|
case None =>
|
||||||
|
if (apiVersion >= 6) {
|
||||||
|
(Errors.GROUP_ID_NOT_FOUND, Some(s"Group $groupId not found."), GroupCoordinator.DeadGroup)
|
||||||
|
} else {
|
||||||
|
(Errors.NONE, None, GroupCoordinator.DeadGroup)
|
||||||
|
}
|
||||||
case Some(group) =>
|
case Some(group) =>
|
||||||
group.inLock {
|
group.inLock {
|
||||||
(Errors.NONE, group.summary)
|
(Errors.NONE, None, group.summary)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -249,10 +249,11 @@ private[group] class GroupCoordinatorAdapter(
|
||||||
): CompletableFuture[util.List[DescribeGroupsResponseData.DescribedGroup]] = {
|
): CompletableFuture[util.List[DescribeGroupsResponseData.DescribedGroup]] = {
|
||||||
|
|
||||||
def describeGroup(groupId: String): DescribeGroupsResponseData.DescribedGroup = {
|
def describeGroup(groupId: String): DescribeGroupsResponseData.DescribedGroup = {
|
||||||
val (error, summary) = coordinator.handleDescribeGroup(groupId)
|
val (error, errorMessage, summary) = coordinator.handleDescribeGroup(groupId, context.apiVersion())
|
||||||
|
|
||||||
new DescribeGroupsResponseData.DescribedGroup()
|
new DescribeGroupsResponseData.DescribedGroup()
|
||||||
.setErrorCode(error.code)
|
.setErrorCode(error.code)
|
||||||
|
.setErrorMessage(errorMessage.orNull)
|
||||||
.setGroupId(groupId)
|
.setGroupId(groupId)
|
||||||
.setGroupState(summary.state)
|
.setGroupState(summary.state)
|
||||||
.setProtocolType(summary.protocolType)
|
.setProtocolType(summary.protocolType)
|
||||||
|
|
|
@ -1610,7 +1610,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
||||||
createTopicWithBrokerPrincipal(topic)
|
createTopicWithBrokerPrincipal(topic)
|
||||||
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), groupResource)
|
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), groupResource)
|
||||||
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource)
|
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource)
|
||||||
createAdminClient().describeConsumerGroups(Seq(group).asJava).describedGroups().get(group).get()
|
val result = createAdminClient().describeConsumerGroups(Seq(group).asJava)
|
||||||
|
JTestUtils.assertFutureThrows(result.describedGroups().get(group), classOf[GroupIdNotFoundException])
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
|
|
|
@ -19,6 +19,7 @@ import org.apache.kafka.common.test.api.ClusterTestExtensions
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import org.apache.kafka.clients.admin.{Admin, ConsumerGroupDescription}
|
import org.apache.kafka.clients.admin.{Admin, ConsumerGroupDescription}
|
||||||
import org.apache.kafka.clients.consumer.{Consumer, GroupProtocol, OffsetAndMetadata}
|
import org.apache.kafka.clients.consumer.{Consumer, GroupProtocol, OffsetAndMetadata}
|
||||||
|
import org.apache.kafka.common.errors.GroupIdNotFoundException
|
||||||
import org.apache.kafka.common.{ConsumerGroupState, GroupType, KafkaFuture, TopicPartition}
|
import org.apache.kafka.common.{ConsumerGroupState, GroupType, KafkaFuture, TopicPartition}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
|
|
||||||
|
@ -221,7 +222,7 @@ class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) {
|
||||||
.asScala
|
.asScala
|
||||||
.toMap
|
.toMap
|
||||||
|
|
||||||
assertDescribedGroup(groups, "grp3", GroupType.CLASSIC, ConsumerGroupState.DEAD)
|
assertDescribedDeadGroup(groups, "grp3")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -328,4 +329,18 @@ class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) {
|
||||||
assertEquals(state, group.state)
|
assertEquals(state, group.state)
|
||||||
assertEquals(Collections.emptyList, group.members)
|
assertEquals(Collections.emptyList, group.members)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private def assertDescribedDeadGroup(
|
||||||
|
groups: Map[String, KafkaFuture[ConsumerGroupDescription]],
|
||||||
|
groupId: String
|
||||||
|
): Unit = {
|
||||||
|
try {
|
||||||
|
groups(groupId).get(10, TimeUnit.SECONDS)
|
||||||
|
fail(s"Group $groupId should not be found")
|
||||||
|
} catch {
|
||||||
|
case e: java.util.concurrent.ExecutionException =>
|
||||||
|
assertTrue(e.getCause.isInstanceOf[GroupIdNotFoundException])
|
||||||
|
assertEquals(s"Group $groupId not found.", e.getCause.getMessage)
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1936,18 +1936,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
||||||
val expectedOperations = AclEntry.supportedOperations(ResourceType.GROUP)
|
val expectedOperations = AclEntry.supportedOperations(ResourceType.GROUP)
|
||||||
assertEquals(expectedOperations, testGroupDescription.authorizedOperations())
|
assertEquals(expectedOperations, testGroupDescription.authorizedOperations())
|
||||||
|
|
||||||
// Test that the fake group is listed as dead.
|
// Test that the fake group throws GroupIdNotFoundException
|
||||||
assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(fakeGroupId))
|
assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(fakeGroupId))
|
||||||
val fakeGroupDescription = describeWithFakeGroupResult.describedGroups().get(fakeGroupId).get()
|
assertFutureThrows(describeWithFakeGroupResult.describedGroups().get(fakeGroupId), classOf[GroupIdNotFoundException],
|
||||||
|
s"Group $fakeGroupId not found.")
|
||||||
|
|
||||||
assertEquals(fakeGroupId, fakeGroupDescription.groupId())
|
// Test that all() also throws GroupIdNotFoundException
|
||||||
assertEquals(0, fakeGroupDescription.members().size())
|
assertFutureThrows(describeWithFakeGroupResult.all(), classOf[GroupIdNotFoundException],
|
||||||
assertEquals("", fakeGroupDescription.partitionAssignor())
|
s"Group $fakeGroupId not found.")
|
||||||
assertEquals(GroupState.DEAD, fakeGroupDescription.groupState())
|
|
||||||
assertEquals(expectedOperations, fakeGroupDescription.authorizedOperations())
|
|
||||||
|
|
||||||
// Test that all() returns 2 results
|
|
||||||
assertEquals(2, describeWithFakeGroupResult.all().get().size())
|
|
||||||
|
|
||||||
val testTopicPart0 = new TopicPartition(testTopicName, 0)
|
val testTopicPart0 = new TopicPartition(testTopicName, 0)
|
||||||
|
|
||||||
|
@ -2209,18 +2205,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
||||||
val expectedOperations = AclEntry.supportedOperations(ResourceType.GROUP)
|
val expectedOperations = AclEntry.supportedOperations(ResourceType.GROUP)
|
||||||
assertEquals(expectedOperations, testGroupDescription.authorizedOperations())
|
assertEquals(expectedOperations, testGroupDescription.authorizedOperations())
|
||||||
|
|
||||||
// Test that the fake group is listed as dead.
|
// Test that the fake group throws GroupIdNotFoundException
|
||||||
assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(fakeGroupId))
|
assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(fakeGroupId))
|
||||||
val fakeGroupDescription = describeWithFakeGroupResult.describedGroups().get(fakeGroupId).get()
|
assertFutureThrows(describeWithFakeGroupResult.describedGroups().get(fakeGroupId),
|
||||||
|
classOf[GroupIdNotFoundException], s"Group $fakeGroupId not found.")
|
||||||
|
|
||||||
assertEquals(fakeGroupId, fakeGroupDescription.groupId())
|
// Test that all() also throws GroupIdNotFoundException
|
||||||
assertEquals(0, fakeGroupDescription.members().size())
|
assertFutureThrows(describeWithFakeGroupResult.all(),
|
||||||
assertEquals("", fakeGroupDescription.partitionAssignor())
|
classOf[GroupIdNotFoundException], s"Group $fakeGroupId not found.")
|
||||||
assertEquals(ConsumerGroupState.DEAD, fakeGroupDescription.state())
|
|
||||||
assertEquals(expectedOperations, fakeGroupDescription.authorizedOperations())
|
|
||||||
|
|
||||||
// Test that all() returns 2 results
|
|
||||||
assertEquals(2, describeWithFakeGroupResult.all().get().size())
|
|
||||||
|
|
||||||
val testTopicPart0 = new TopicPartition(testTopicName, 0)
|
val testTopicPart0 = new TopicPartition(testTopicName, 0)
|
||||||
|
|
||||||
|
@ -2642,17 +2634,14 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
||||||
val expectedOperations = AclEntry.supportedOperations(ResourceType.GROUP)
|
val expectedOperations = AclEntry.supportedOperations(ResourceType.GROUP)
|
||||||
assertEquals(expectedOperations, testGroupDescription.authorizedOperations())
|
assertEquals(expectedOperations, testGroupDescription.authorizedOperations())
|
||||||
|
|
||||||
// Test that the fake group is listed as dead.
|
// Test that the fake group throws GroupIdNotFoundException
|
||||||
assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(fakeGroupId))
|
assertTrue(describeWithFakeGroupResult.describedGroups().containsKey(fakeGroupId))
|
||||||
val fakeGroupDescription = describeWithFakeGroupResult.describedGroups().get(fakeGroupId).get()
|
assertFutureThrows(describeWithFakeGroupResult.describedGroups().get(fakeGroupId),
|
||||||
|
classOf[GroupIdNotFoundException], s"Group $fakeGroupId not found.")
|
||||||
|
|
||||||
assertEquals(fakeGroupId, fakeGroupDescription.groupId())
|
// Test that all() also throws GroupIdNotFoundException
|
||||||
assertEquals(0, fakeGroupDescription.members().size())
|
assertFutureThrows(describeWithFakeGroupResult.all(),
|
||||||
assertEquals(GroupState.DEAD, fakeGroupDescription.groupState())
|
classOf[GroupIdNotFoundException], s"Group $fakeGroupId not found.")
|
||||||
assertNull(fakeGroupDescription.authorizedOperations())
|
|
||||||
|
|
||||||
// Test that all() returns 2 results
|
|
||||||
assertEquals(2, describeWithFakeGroupResult.all().get().size())
|
|
||||||
|
|
||||||
val describeTestGroupResult = client.describeShareGroups(Collections.singleton(testGroupId),
|
val describeTestGroupResult = client.describeShareGroups(Collections.singleton(testGroupId),
|
||||||
new DescribeShareGroupsOptions().includeAuthorizedOperations(true))
|
new DescribeShareGroupsOptions().includeAuthorizedOperations(true))
|
||||||
|
@ -2664,18 +2653,12 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
||||||
assertEquals(testGroupId, testGroupDescription.groupId)
|
assertEquals(testGroupId, testGroupDescription.groupId)
|
||||||
assertEquals(consumerSet.size, testGroupDescription.members().size())
|
assertEquals(consumerSet.size, testGroupDescription.members().size())
|
||||||
|
|
||||||
// Describing a share group using describeConsumerGroups reports it as a DEAD consumer group
|
// Describing a share group using describeConsumerGroups reports it as a non-existent group
|
||||||
// in the same way as a non-existent group
|
// but the error message is different
|
||||||
val describeConsumerGroupResult = client.describeConsumerGroups(Collections.singleton(testGroupId),
|
val describeConsumerGroupResult = client.describeConsumerGroups(Collections.singleton(testGroupId),
|
||||||
new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true))
|
new DescribeConsumerGroupsOptions().includeAuthorizedOperations(true))
|
||||||
assertEquals(1, describeConsumerGroupResult.all().get().size())
|
assertFutureThrows(describeConsumerGroupResult.all(),
|
||||||
|
classOf[GroupIdNotFoundException], s"Group $testGroupId is not a consumer group.")
|
||||||
val deadConsumerGroupDescription = describeConsumerGroupResult.describedGroups().get(testGroupId).get()
|
|
||||||
assertEquals(testGroupId, deadConsumerGroupDescription.groupId())
|
|
||||||
assertEquals(0, deadConsumerGroupDescription.members().size())
|
|
||||||
assertEquals("", deadConsumerGroupDescription.partitionAssignor())
|
|
||||||
assertEquals(ConsumerGroupState.DEAD, deadConsumerGroupDescription.state())
|
|
||||||
assertEquals(expectedOperations, deadConsumerGroupDescription.authorizedOperations())
|
|
||||||
} finally {
|
} finally {
|
||||||
consumerThreads.foreach {
|
consumerThreads.foreach {
|
||||||
case consumerThread =>
|
case consumerThread =>
|
||||||
|
|
|
@ -414,12 +414,12 @@ class GroupCoordinatorAdapterTest {
|
||||||
))
|
))
|
||||||
)
|
)
|
||||||
|
|
||||||
when(groupCoordinator.handleDescribeGroup(groupId1)).thenReturn {
|
when(groupCoordinator.handleDescribeGroup(groupId1, ApiKeys.DESCRIBE_GROUPS.latestVersion)).thenReturn {
|
||||||
(Errors.NONE, groupSummary1)
|
(Errors.NONE, None, groupSummary1)
|
||||||
}
|
}
|
||||||
|
|
||||||
when(groupCoordinator.handleDescribeGroup(groupId2)).thenReturn {
|
when(groupCoordinator.handleDescribeGroup(groupId2, ApiKeys.DESCRIBE_GROUPS.latestVersion)).thenReturn {
|
||||||
(Errors.NOT_COORDINATOR, GroupCoordinator.EmptyGroup)
|
(Errors.NOT_COORDINATOR, None, GroupCoordinator.EmptyGroup)
|
||||||
}
|
}
|
||||||
|
|
||||||
val ctx = makeContext(ApiKeys.DESCRIBE_GROUPS, ApiKeys.DESCRIBE_GROUPS.latestVersion)
|
val ctx = makeContext(ApiKeys.DESCRIBE_GROUPS, ApiKeys.DESCRIBE_GROUPS.latestVersion)
|
||||||
|
|
|
@ -175,7 +175,7 @@ class GroupCoordinatorTest {
|
||||||
assertEquals(Some(Errors.NONE), heartbeatError)
|
assertEquals(Some(Errors.NONE), heartbeatError)
|
||||||
|
|
||||||
// DescribeGroups
|
// DescribeGroups
|
||||||
val (describeGroupError, _) = groupCoordinator.handleDescribeGroup(otherGroupId)
|
val (describeGroupError, _, _) = groupCoordinator.handleDescribeGroup(otherGroupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)
|
||||||
assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS, describeGroupError)
|
assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS, describeGroupError)
|
||||||
|
|
||||||
// ListGroups
|
// ListGroups
|
||||||
|
@ -187,15 +187,16 @@ class GroupCoordinatorTest {
|
||||||
assertEquals(Some(Errors.COORDINATOR_LOAD_IN_PROGRESS), deleteGroupsErrors.get(otherGroupId))
|
assertEquals(Some(Errors.COORDINATOR_LOAD_IN_PROGRESS), deleteGroupsErrors.get(otherGroupId))
|
||||||
|
|
||||||
// Check that non-loading groups are still accessible
|
// Check that non-loading groups are still accessible
|
||||||
assertEquals(Errors.NONE, groupCoordinator.handleDescribeGroup(groupId)._1)
|
assertEquals(Errors.GROUP_ID_NOT_FOUND, groupCoordinator.handleDescribeGroup(groupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)._1)
|
||||||
|
|
||||||
// After loading, we should be able to access the group
|
// After loading, we should be able to access the group
|
||||||
val otherGroupMetadataTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, otherGroupPartitionId)
|
val otherGroupMetadataTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, otherGroupPartitionId)
|
||||||
when(replicaManager.getLog(otherGroupMetadataTopicPartition)).thenReturn(None)
|
when(replicaManager.getLog(otherGroupMetadataTopicPartition)).thenReturn(None)
|
||||||
|
|
||||||
// Call removeGroupsAndOffsets so that partition removed from loadingPartitions
|
// Call removeGroupsAndOffsets so that partition removed from loadingPartitions
|
||||||
groupCoordinator.groupManager.removeGroupsAndOffsets(otherGroupMetadataTopicPartition, OptionalInt.of(1), group => {})
|
groupCoordinator.groupManager.removeGroupsAndOffsets(otherGroupMetadataTopicPartition, OptionalInt.of(1), group => {})
|
||||||
groupCoordinator.groupManager.loadGroupsAndOffsets(otherGroupMetadataTopicPartition, 1, group => {}, 0L)
|
groupCoordinator.groupManager.loadGroupsAndOffsets(otherGroupMetadataTopicPartition, 1, group => {}, 0L)
|
||||||
assertEquals(Errors.NONE, groupCoordinator.handleDescribeGroup(otherGroupId)._1)
|
assertEquals(Errors.GROUP_ID_NOT_FOUND, groupCoordinator.handleDescribeGroup(otherGroupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)._1)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -2609,8 +2610,9 @@ class GroupCoordinatorTest {
|
||||||
assertEquals(Errors.NONE, fetchError)
|
assertEquals(Errors.NONE, fetchError)
|
||||||
assertEquals(Some(0), partitionData.get(tip.topicPartition).map(_.offset))
|
assertEquals(Some(0), partitionData.get(tip.topicPartition).map(_.offset))
|
||||||
|
|
||||||
val (describeError, summary) = groupCoordinator.handleDescribeGroup(groupId)
|
var (describeError, describeErrorMessage, summary) = groupCoordinator.handleDescribeGroup(groupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)
|
||||||
assertEquals(Errors.NONE, describeError)
|
assertEquals(Errors.NONE, describeError)
|
||||||
|
assertTrue(describeErrorMessage.isEmpty)
|
||||||
assertEquals(Empty.toString, summary.state)
|
assertEquals(Empty.toString, summary.state)
|
||||||
|
|
||||||
val groupTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)
|
val groupTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)
|
||||||
|
@ -3405,15 +3407,21 @@ class GroupCoordinatorTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testDescribeGroupWrongCoordinator(): Unit = {
|
def testDescribeGroupWrongCoordinator(): Unit = {
|
||||||
val (error, _) = groupCoordinator.handleDescribeGroup(otherGroupId)
|
val (error, _, _) = groupCoordinator.handleDescribeGroup(otherGroupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)
|
||||||
assertEquals(Errors.NOT_COORDINATOR, error)
|
assertEquals(Errors.NOT_COORDINATOR, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testDescribeGroupInactiveGroup(): Unit = {
|
def testDescribeGroupInactiveGroup(): Unit = {
|
||||||
val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
|
val (error, errorMessage, summary) = groupCoordinator.handleDescribeGroup(groupId, 5)
|
||||||
assertEquals(Errors.NONE, error)
|
assertEquals(Errors.NONE, error)
|
||||||
|
assertTrue(errorMessage.isEmpty)
|
||||||
assertEquals(GroupCoordinator.DeadGroup, summary)
|
assertEquals(GroupCoordinator.DeadGroup, summary)
|
||||||
|
|
||||||
|
val (errorV6, errorMessageV6, summaryV6) = groupCoordinator.handleDescribeGroup(groupId, 6)
|
||||||
|
assertEquals(Errors.GROUP_ID_NOT_FOUND, errorV6)
|
||||||
|
assertEquals(s"Group $groupId not found.", errorMessageV6.get)
|
||||||
|
assertEquals(GroupCoordinator.DeadGroup, summaryV6)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -3427,8 +3435,9 @@ class GroupCoordinatorTest {
|
||||||
val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
|
val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
|
||||||
assertEquals(Errors.NONE, syncGroupResult.error)
|
assertEquals(Errors.NONE, syncGroupResult.error)
|
||||||
|
|
||||||
val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
|
val (error, errorMessage, summary) = groupCoordinator.handleDescribeGroup(groupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)
|
||||||
assertEquals(Errors.NONE, error)
|
assertEquals(Errors.NONE, error)
|
||||||
|
assertTrue(errorMessage.isEmpty)
|
||||||
assertEquals(protocolType, summary.protocolType)
|
assertEquals(protocolType, summary.protocolType)
|
||||||
assertEquals("range", summary.protocol)
|
assertEquals("range", summary.protocol)
|
||||||
assertEquals(List(assignedMemberId), summary.members.map(_.memberId))
|
assertEquals(List(assignedMemberId), summary.members.map(_.memberId))
|
||||||
|
@ -3445,8 +3454,9 @@ class GroupCoordinatorTest {
|
||||||
val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
|
val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
|
||||||
assertEquals(Errors.NONE, syncGroupResult.error)
|
assertEquals(Errors.NONE, syncGroupResult.error)
|
||||||
|
|
||||||
val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
|
val (error, errorMessage, summary) = groupCoordinator.handleDescribeGroup(groupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)
|
||||||
assertEquals(Errors.NONE, error)
|
assertEquals(Errors.NONE, error)
|
||||||
|
assertTrue(errorMessage.isEmpty)
|
||||||
assertEquals(protocolType, summary.protocolType)
|
assertEquals(protocolType, summary.protocolType)
|
||||||
assertEquals("range", summary.protocol)
|
assertEquals("range", summary.protocol)
|
||||||
assertEquals(List(assignedMemberId), summary.members.map(_.memberId))
|
assertEquals(List(assignedMemberId), summary.members.map(_.memberId))
|
||||||
|
@ -3460,8 +3470,9 @@ class GroupCoordinatorTest {
|
||||||
val joinGroupError = joinGroupResult.error
|
val joinGroupError = joinGroupResult.error
|
||||||
assertEquals(Errors.NONE, joinGroupError)
|
assertEquals(Errors.NONE, joinGroupError)
|
||||||
|
|
||||||
val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
|
val (error, errorMessage, summary) = groupCoordinator.handleDescribeGroup(groupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)
|
||||||
assertEquals(Errors.NONE, error)
|
assertEquals(Errors.NONE, error)
|
||||||
|
assertTrue(errorMessage.isEmpty)
|
||||||
assertEquals(protocolType, summary.protocolType)
|
assertEquals(protocolType, summary.protocolType)
|
||||||
assertEquals(GroupCoordinator.NoProtocol, summary.protocol)
|
assertEquals(GroupCoordinator.NoProtocol, summary.protocol)
|
||||||
assertEquals(CompletingRebalance.toString, summary.state)
|
assertEquals(CompletingRebalance.toString, summary.state)
|
||||||
|
@ -3528,9 +3539,9 @@ class GroupCoordinatorTest {
|
||||||
val commitOffsetResult = commitOffsets(groupId, assignedMemberId, joinGroupResult.generationId, Map(tip -> offset))
|
val commitOffsetResult = commitOffsets(groupId, assignedMemberId, joinGroupResult.generationId, Map(tip -> offset))
|
||||||
assertEquals(Map(tip -> Errors.NONE), commitOffsetResult)
|
assertEquals(Map(tip -> Errors.NONE), commitOffsetResult)
|
||||||
|
|
||||||
val describeGroupResult = groupCoordinator.handleDescribeGroup(groupId)
|
val describeGroupResult = groupCoordinator.handleDescribeGroup(groupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)
|
||||||
assertEquals(Stable.toString, describeGroupResult._2.state)
|
assertEquals(Stable.toString, describeGroupResult._3.state)
|
||||||
assertEquals(assignedMemberId, describeGroupResult._2.members.head.memberId)
|
assertEquals(assignedMemberId, describeGroupResult._3.members.head.memberId)
|
||||||
|
|
||||||
val leaveGroupResults = singleLeaveGroup(groupId, assignedMemberId)
|
val leaveGroupResults = singleLeaveGroup(groupId, assignedMemberId)
|
||||||
verifyLeaveGroupResult(leaveGroupResults)
|
verifyLeaveGroupResult(leaveGroupResults)
|
||||||
|
@ -3545,7 +3556,7 @@ class GroupCoordinatorTest {
|
||||||
val result = groupCoordinator.handleDeleteGroups(Set(groupId))
|
val result = groupCoordinator.handleDeleteGroups(Set(groupId))
|
||||||
assert(result.size == 1 && result.contains(groupId) && result.get(groupId).contains(Errors.NONE))
|
assert(result.size == 1 && result.contains(groupId) && result.get(groupId).contains(Errors.NONE))
|
||||||
|
|
||||||
assertEquals(Dead.toString, groupCoordinator.handleDescribeGroup(groupId)._2.state)
|
assertEquals(Dead.toString, groupCoordinator.handleDescribeGroup(groupId, ApiKeys.DESCRIBE_GROUPS.latestVersion)._3.state)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -123,6 +123,8 @@ class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinator
|
||||||
List(new DescribedGroup()
|
List(new DescribedGroup()
|
||||||
.setGroupId("grp")
|
.setGroupId("grp")
|
||||||
.setGroupState(ClassicGroupState.DEAD.toString)
|
.setGroupState(ClassicGroupState.DEAD.toString)
|
||||||
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code)
|
||||||
|
.setErrorMessage("Group grp not found.")
|
||||||
),
|
),
|
||||||
describeGroups(List("grp"))
|
describeGroups(List("grp"))
|
||||||
)
|
)
|
||||||
|
|
|
@ -20,7 +20,7 @@ import org.apache.kafka.common.test.api.ClusterInstance
|
||||||
import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type}
|
import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type}
|
||||||
import org.apache.kafka.common.test.api.ClusterTestExtensions
|
import org.apache.kafka.common.test.api.ClusterTestExtensions
|
||||||
import org.apache.kafka.common.message.DescribeGroupsResponseData.{DescribedGroup, DescribedGroupMember}
|
import org.apache.kafka.common.message.DescribeGroupsResponseData.{DescribedGroup, DescribedGroupMember}
|
||||||
import org.apache.kafka.common.protocol.ApiKeys
|
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||||
import org.apache.kafka.coordinator.group.classic.ClassicGroupState
|
import org.apache.kafka.coordinator.group.classic.ClassicGroupState
|
||||||
import org.junit.jupiter.api.Assertions.assertEquals
|
import org.junit.jupiter.api.Assertions.assertEquals
|
||||||
|
@ -106,6 +106,8 @@ class DescribeGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinat
|
||||||
new DescribedGroup()
|
new DescribedGroup()
|
||||||
.setGroupId("grp-unknown")
|
.setGroupId("grp-unknown")
|
||||||
.setGroupState(ClassicGroupState.DEAD.toString) // Return DEAD group when the group does not exist.
|
.setGroupState(ClassicGroupState.DEAD.toString) // Return DEAD group when the group does not exist.
|
||||||
|
.setErrorCode(if (version >= 6) Errors.GROUP_ID_NOT_FOUND.code() else Errors.NONE.code())
|
||||||
|
.setErrorMessage(if (version >= 6) "Group grp-unknown not found." else null)
|
||||||
),
|
),
|
||||||
describeGroups(
|
describeGroups(
|
||||||
groupIds = List("grp-1", "grp-2", "grp-unknown"),
|
groupIds = List("grp-1", "grp-2", "grp-unknown"),
|
||||||
|
|
|
@ -3719,7 +3719,8 @@ class KafkaApisTest extends Logging {
|
||||||
val describeGroupsRequest = new DescribeGroupsRequestData().setGroups(List(
|
val describeGroupsRequest = new DescribeGroupsRequestData().setGroups(List(
|
||||||
"group-1",
|
"group-1",
|
||||||
"group-2",
|
"group-2",
|
||||||
"group-3"
|
"group-3",
|
||||||
|
"group-4"
|
||||||
).asJava)
|
).asJava)
|
||||||
|
|
||||||
val requestChannelRequest = buildRequest(new DescribeGroupsRequest.Builder(describeGroupsRequest).build())
|
val requestChannelRequest = buildRequest(new DescribeGroupsRequest.Builder(describeGroupsRequest).build())
|
||||||
|
@ -3746,7 +3747,12 @@ class KafkaApisTest extends Logging {
|
||||||
.setErrorCode(Errors.NOT_COORDINATOR.code),
|
.setErrorCode(Errors.NOT_COORDINATOR.code),
|
||||||
new DescribeGroupsResponseData.DescribedGroup()
|
new DescribeGroupsResponseData.DescribedGroup()
|
||||||
.setGroupId("group-3")
|
.setGroupId("group-3")
|
||||||
.setErrorCode(Errors.REQUEST_TIMED_OUT.code)
|
.setErrorCode(Errors.REQUEST_TIMED_OUT.code),
|
||||||
|
new DescribeGroupsResponseData.DescribedGroup()
|
||||||
|
.setGroupId("group-4")
|
||||||
|
.setGroupState("Dead")
|
||||||
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code)
|
||||||
|
.setErrorMessage("Group group-4 is not a classic group.")
|
||||||
).asJava
|
).asJava
|
||||||
|
|
||||||
future.complete(groupResults)
|
future.complete(groupResults)
|
||||||
|
|
|
@ -161,12 +161,19 @@
|
||||||
</li>
|
</li>
|
||||||
</ul>
|
</ul>
|
||||||
</li>
|
</li>
|
||||||
<li><b>Admin</b>
|
<li><b>Admin client</b>
|
||||||
<ul>
|
<ul>
|
||||||
<li>
|
<li>
|
||||||
The <code>alterConfigs</code> method was removed from the <code>org.apache.kafka.clients.admin.Admin</code>
|
The <code>alterConfigs</code> method was removed from the <code>org.apache.kafka.clients.admin.Admin</code>.
|
||||||
Please use <code>incrementalAlterConfigs</code> instead.
|
Please use <code>incrementalAlterConfigs</code> instead.
|
||||||
</li>
|
</li>
|
||||||
|
<li>The <code>org.apache.kafka.common.ConsumerGroupState</code> enumeration and related methods have been deprecated. Please use <code>GroupState</code> instead
|
||||||
|
which applies to all types of group.
|
||||||
|
</li>
|
||||||
|
<li>The <code>Admin.describeConsumerGroups</code> method used to return a <code>ConsumerGroupDescription</code> in state
|
||||||
|
<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>
|
||||||
</ul>
|
</ul>
|
||||||
</li>
|
</li>
|
||||||
</ul>
|
</ul>
|
||||||
|
@ -185,10 +192,11 @@
|
||||||
See <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=181308223">KIP-750</a> for more details
|
See <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=181308223">KIP-750</a> for more details
|
||||||
</li>
|
</li>
|
||||||
<li>
|
<li>
|
||||||
KafkaLog4jAppender has been remove, users should migrate to the log4j2 appender
|
KafkaLog4jAppender has been removed, users should migrate to the log4j2 appender
|
||||||
See <a href="https://logging.apache.org/log4j/2.x/manual/appenders.html#KafkaAppender">KafkaAppender</a> for more details
|
See <a href="https://logging.apache.org/log4j/2.x/manual/appenders.html#KafkaAppender">KafkaAppender</a> for more details
|
||||||
</li>
|
</li>
|
||||||
<li>The <code>--delete-config</code> option in the <code>kafka-topics</code> command line tool has been deprecated.
|
<li>
|
||||||
|
The <code>--delete-config</code> option in the <code>kafka-topics</code> command line tool has been deprecated.
|
||||||
</li>
|
</li>
|
||||||
<li>
|
<li>
|
||||||
For implementors of RemoteLogMetadataManager (RLMM), a new API `nextSegmentWithTxnIndex` is
|
For implementors of RemoteLogMetadataManager (RLMM), a new API `nextSegmentWithTxnIndex` is
|
||||||
|
|
|
@ -603,7 +603,7 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
|
||||||
List<String> groupIds,
|
List<String> groupIds,
|
||||||
long committedOffset
|
long committedOffset
|
||||||
) {
|
) {
|
||||||
return groupMetadataManager.describeGroups(groupIds, committedOffset);
|
return groupMetadataManager.describeGroups(context, groupIds, committedOffset);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -521,6 +521,7 @@ public class GroupMetadataManager {
|
||||||
describedGroups.add(new ConsumerGroupDescribeResponseData.DescribedGroup()
|
describedGroups.add(new ConsumerGroupDescribeResponseData.DescribedGroup()
|
||||||
.setGroupId(groupId)
|
.setGroupId(groupId)
|
||||||
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
||||||
|
.setErrorMessage(exception.getMessage())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -552,6 +553,7 @@ public class GroupMetadataManager {
|
||||||
describedGroups.add(new ShareGroupDescribeResponseData.DescribedGroup()
|
describedGroups.add(new ShareGroupDescribeResponseData.DescribedGroup()
|
||||||
.setGroupId(groupId)
|
.setGroupId(groupId)
|
||||||
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
||||||
|
.setErrorMessage(exception.getMessage())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -562,12 +564,14 @@ public class GroupMetadataManager {
|
||||||
/**
|
/**
|
||||||
* Handles a DescribeGroup request.
|
* Handles a DescribeGroup request.
|
||||||
*
|
*
|
||||||
|
* @param context The request context.
|
||||||
* @param groupIds The IDs of the groups to describe.
|
* @param groupIds The IDs of the groups to describe.
|
||||||
* @param committedOffset A specified committed offset corresponding to this shard.
|
* @param committedOffset A specified committed offset corresponding to this shard.
|
||||||
*
|
*
|
||||||
* @return A list containing the DescribeGroupsResponseData.DescribedGroup.
|
* @return A list containing the DescribeGroupsResponseData.DescribedGroup.
|
||||||
*/
|
*/
|
||||||
public List<DescribeGroupsResponseData.DescribedGroup> describeGroups(
|
public List<DescribeGroupsResponseData.DescribedGroup> describeGroups(
|
||||||
|
RequestContext context,
|
||||||
List<String> groupIds,
|
List<String> groupIds,
|
||||||
long committedOffset
|
long committedOffset
|
||||||
) {
|
) {
|
||||||
|
@ -603,10 +607,19 @@ public class GroupMetadataManager {
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
} catch (GroupIdNotFoundException exception) {
|
} catch (GroupIdNotFoundException exception) {
|
||||||
describedGroups.add(new DescribeGroupsResponseData.DescribedGroup()
|
if (context.header.apiVersion() >= 6) {
|
||||||
.setGroupId(groupId)
|
describedGroups.add(new DescribeGroupsResponseData.DescribedGroup()
|
||||||
.setGroupState(DEAD.toString())
|
.setGroupId(groupId)
|
||||||
);
|
.setGroupState(DEAD.toString())
|
||||||
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
||||||
|
.setErrorMessage(exception.getMessage())
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
describedGroups.add(new DescribeGroupsResponseData.DescribedGroup()
|
||||||
|
.setGroupId(groupId)
|
||||||
|
.setGroupState(DEAD.toString())
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
return describedGroups;
|
return describedGroups;
|
||||||
|
@ -647,7 +660,7 @@ public class GroupMetadataManager {
|
||||||
} else if (createIfNotExists && group.type() == CLASSIC && validateOnlineUpgrade((ClassicGroup) group)) {
|
} else if (createIfNotExists && group.type() == CLASSIC && validateOnlineUpgrade((ClassicGroup) group)) {
|
||||||
return convertToConsumerGroup((ClassicGroup) group, records);
|
return convertToConsumerGroup((ClassicGroup) group, records);
|
||||||
} else {
|
} else {
|
||||||
throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group", groupId));
|
throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group.", groupId));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -670,7 +683,7 @@ public class GroupMetadataManager {
|
||||||
if (group.type() == CONSUMER) {
|
if (group.type() == CONSUMER) {
|
||||||
return (ConsumerGroup) group;
|
return (ConsumerGroup) group;
|
||||||
} else {
|
} else {
|
||||||
throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group", groupId));
|
throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group.", groupId));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -704,7 +717,7 @@ public class GroupMetadataManager {
|
||||||
Group group = groups.get(groupId);
|
Group group = groups.get(groupId);
|
||||||
|
|
||||||
if (group == null && !createIfNotExists) {
|
if (group == null && !createIfNotExists) {
|
||||||
throw new GroupIdNotFoundException(String.format("Consumer group %s not found", groupId));
|
throw new GroupIdNotFoundException(String.format("Consumer group %s not found.", groupId));
|
||||||
}
|
}
|
||||||
|
|
||||||
if (group == null) {
|
if (group == null) {
|
||||||
|
|
|
@ -8699,7 +8699,8 @@ public class GroupMetadataManagerTest {
|
||||||
List<ConsumerGroupDescribeResponseData.DescribedGroup> actual = context.sendConsumerGroupDescribe(List.of(groupId));
|
List<ConsumerGroupDescribeResponseData.DescribedGroup> actual = context.sendConsumerGroupDescribe(List.of(groupId));
|
||||||
ConsumerGroupDescribeResponseData.DescribedGroup describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup()
|
ConsumerGroupDescribeResponseData.DescribedGroup describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup()
|
||||||
.setGroupId(groupId)
|
.setGroupId(groupId)
|
||||||
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code());
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
||||||
|
.setErrorMessage("Group " + groupId + " not found.");
|
||||||
List<ConsumerGroupDescribeResponseData.DescribedGroup> expected = List.of(
|
List<ConsumerGroupDescribeResponseData.DescribedGroup> expected = List.of(
|
||||||
describedGroup
|
describedGroup
|
||||||
);
|
);
|
||||||
|
@ -8741,7 +8742,8 @@ public class GroupMetadataManagerTest {
|
||||||
List<ConsumerGroupDescribeResponseData.DescribedGroup> actual = context.groupMetadataManager.consumerGroupDescribe(List.of(consumerGroupId), context.lastCommittedOffset);
|
List<ConsumerGroupDescribeResponseData.DescribedGroup> actual = context.groupMetadataManager.consumerGroupDescribe(List.of(consumerGroupId), context.lastCommittedOffset);
|
||||||
ConsumerGroupDescribeResponseData.DescribedGroup describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup()
|
ConsumerGroupDescribeResponseData.DescribedGroup describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup()
|
||||||
.setGroupId(consumerGroupId)
|
.setGroupId(consumerGroupId)
|
||||||
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code());
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
||||||
|
.setErrorMessage("Group " + consumerGroupId + " not found.");
|
||||||
List<ConsumerGroupDescribeResponseData.DescribedGroup> expected = List.of(
|
List<ConsumerGroupDescribeResponseData.DescribedGroup> expected = List.of(
|
||||||
describedGroup
|
describedGroup
|
||||||
);
|
);
|
||||||
|
@ -8873,6 +8875,13 @@ public class GroupMetadataManagerTest {
|
||||||
context.verifyDescribeGroupsReturnsDeadGroup("group-id");
|
context.verifyDescribeGroupsReturnsDeadGroup("group-id");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testDescribeGroupsBeforeV6GroupIdNotFoundException() {
|
||||||
|
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
|
||||||
|
.build();
|
||||||
|
context.verifyDescribeGroupsBeforeV6ReturnsDeadGroup("group-id");
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testGroupStuckInRebalanceTimeoutDueToNonjoinedStaticMember() throws Exception {
|
public void testGroupStuckInRebalanceTimeoutDueToNonjoinedStaticMember() throws Exception {
|
||||||
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
|
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
|
||||||
|
@ -14976,6 +14985,7 @@ public class GroupMetadataManagerTest {
|
||||||
new ConsumerGroupDescribeResponseData.DescribedGroup()
|
new ConsumerGroupDescribeResponseData.DescribedGroup()
|
||||||
.setGroupId(groupId)
|
.setGroupId(groupId)
|
||||||
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
||||||
|
.setErrorMessage("Group " + groupId + " is not a consumer group.")
|
||||||
);
|
);
|
||||||
|
|
||||||
List<ConsumerGroupDescribeResponseData.DescribedGroup> actual = context.sendConsumerGroupDescribe(List.of(groupId));
|
List<ConsumerGroupDescribeResponseData.DescribedGroup> actual = context.sendConsumerGroupDescribe(List.of(groupId));
|
||||||
|
@ -15048,6 +15058,7 @@ public class GroupMetadataManagerTest {
|
||||||
new ShareGroupDescribeResponseData.DescribedGroup()
|
new ShareGroupDescribeResponseData.DescribedGroup()
|
||||||
.setGroupId(groupId)
|
.setGroupId(groupId)
|
||||||
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
||||||
|
.setErrorMessage("Group " + groupId + " is not a share group.")
|
||||||
);
|
);
|
||||||
|
|
||||||
List<ShareGroupDescribeResponseData.DescribedGroup> actual = context.sendShareGroupDescribe(List.of(groupId));
|
List<ShareGroupDescribeResponseData.DescribedGroup> actual = context.sendShareGroupDescribe(List.of(groupId));
|
||||||
|
|
|
@ -1270,7 +1270,43 @@ public class GroupMetadataManagerTestContext {
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<DescribeGroupsResponseData.DescribedGroup> describeGroups(List<String> groupIds) {
|
public List<DescribeGroupsResponseData.DescribedGroup> describeGroups(List<String> groupIds) {
|
||||||
return groupMetadataManager.describeGroups(groupIds, lastCommittedOffset);
|
RequestContext context = new RequestContext(
|
||||||
|
new RequestHeader(
|
||||||
|
ApiKeys.DESCRIBE_GROUPS,
|
||||||
|
ApiKeys.DESCRIBE_GROUPS.latestVersion(),
|
||||||
|
DEFAULT_CLIENT_ID,
|
||||||
|
0
|
||||||
|
),
|
||||||
|
"1",
|
||||||
|
DEFAULT_CLIENT_ADDRESS,
|
||||||
|
KafkaPrincipal.ANONYMOUS,
|
||||||
|
ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
|
||||||
|
SecurityProtocol.PLAINTEXT,
|
||||||
|
ClientInformation.EMPTY,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
|
||||||
|
return groupMetadataManager.describeGroups(context, groupIds, lastCommittedOffset);
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<DescribeGroupsResponseData.DescribedGroup> describeGroups(List<String> groupIds, short apiVersion) {
|
||||||
|
RequestContext context = new RequestContext(
|
||||||
|
new RequestHeader(
|
||||||
|
ApiKeys.DESCRIBE_GROUPS,
|
||||||
|
apiVersion,
|
||||||
|
DEFAULT_CLIENT_ID,
|
||||||
|
0
|
||||||
|
),
|
||||||
|
"1",
|
||||||
|
DEFAULT_CLIENT_ADDRESS,
|
||||||
|
KafkaPrincipal.ANONYMOUS,
|
||||||
|
ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
|
||||||
|
SecurityProtocol.PLAINTEXT,
|
||||||
|
ClientInformation.EMPTY,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
|
||||||
|
return groupMetadataManager.describeGroups(context, groupIds, lastCommittedOffset);
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<ShareGroupDescribeResponseData.DescribedGroup> sendShareGroupDescribe(List<String> groupIds) {
|
public List<ShareGroupDescribeResponseData.DescribedGroup> sendShareGroupDescribe(List<String> groupIds) {
|
||||||
|
@ -1390,6 +1426,21 @@ public class GroupMetadataManagerTestContext {
|
||||||
List.of(new DescribeGroupsResponseData.DescribedGroup()
|
List.of(new DescribeGroupsResponseData.DescribedGroup()
|
||||||
.setGroupId(groupId)
|
.setGroupId(groupId)
|
||||||
.setGroupState(DEAD.toString())
|
.setGroupState(DEAD.toString())
|
||||||
|
.setErrorCode(Errors.GROUP_ID_NOT_FOUND.code())
|
||||||
|
.setErrorMessage("Group " + groupId + " not found.")
|
||||||
|
),
|
||||||
|
describedGroups
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void verifyDescribeGroupsBeforeV6ReturnsDeadGroup(String groupId) {
|
||||||
|
List<DescribeGroupsResponseData.DescribedGroup> describedGroups =
|
||||||
|
describeGroups(Collections.singletonList(groupId), (short) 5);
|
||||||
|
|
||||||
|
assertEquals(
|
||||||
|
Collections.singletonList(new DescribeGroupsResponseData.DescribedGroup()
|
||||||
|
.setGroupId(groupId)
|
||||||
|
.setGroupState(DEAD.toString())
|
||||||
),
|
),
|
||||||
describedGroups
|
describedGroups
|
||||||
);
|
);
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.kafka.clients.producer.ProducerRecord;
|
||||||
import org.apache.kafka.clients.producer.RecordMetadata;
|
import org.apache.kafka.clients.producer.RecordMetadata;
|
||||||
import org.apache.kafka.common.Metric;
|
import org.apache.kafka.common.Metric;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
||||||
import org.apache.kafka.common.errors.TimeoutException;
|
import org.apache.kafka.common.errors.TimeoutException;
|
||||||
import org.apache.kafka.common.header.Headers;
|
import org.apache.kafka.common.header.Headers;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
|
@ -1002,7 +1003,9 @@ public class IntegrationTestUtils {
|
||||||
.get(applicationId)
|
.get(applicationId)
|
||||||
.get();
|
.get();
|
||||||
return groupDescription.members().isEmpty();
|
return groupDescription.members().isEmpty();
|
||||||
} catch (final ExecutionException | InterruptedException e) {
|
} catch (final ExecutionException e) {
|
||||||
|
return e.getCause() instanceof GroupIdNotFoundException;
|
||||||
|
} catch (final InterruptedException e) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
|
||||||
import org.apache.kafka.common.KafkaFuture;
|
import org.apache.kafka.common.KafkaFuture;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.annotation.InterfaceStability;
|
import org.apache.kafka.common.annotation.InterfaceStability;
|
||||||
|
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
||||||
import org.apache.kafka.common.requests.ListOffsetsResponse;
|
import org.apache.kafka.common.requests.ListOffsetsResponse;
|
||||||
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
|
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
|
||||||
import org.apache.kafka.common.utils.Exit;
|
import org.apache.kafka.common.utils.Exit;
|
||||||
|
@ -170,17 +171,24 @@ public class StreamsResetter {
|
||||||
final DescribeConsumerGroupsResult describeResult = adminClient.describeConsumerGroups(
|
final DescribeConsumerGroupsResult describeResult = adminClient.describeConsumerGroups(
|
||||||
Collections.singleton(groupId),
|
Collections.singleton(groupId),
|
||||||
new DescribeConsumerGroupsOptions().timeoutMs(10 * 1000));
|
new DescribeConsumerGroupsOptions().timeoutMs(10 * 1000));
|
||||||
final List<MemberDescription> members =
|
try {
|
||||||
new ArrayList<>(describeResult.describedGroups().get(groupId).get().members());
|
final List<MemberDescription> members =
|
||||||
if (!members.isEmpty()) {
|
new ArrayList<>(describeResult.describedGroups().get(groupId).get().members());
|
||||||
if (options.hasForce()) {
|
if (!members.isEmpty()) {
|
||||||
System.out.println("Force deleting all active members in the group: " + groupId);
|
if (options.hasForce()) {
|
||||||
adminClient.removeMembersFromConsumerGroup(groupId, new RemoveMembersFromConsumerGroupOptions()).all().get();
|
System.out.println("Force deleting all active members in the group: " + groupId);
|
||||||
} else {
|
adminClient.removeMembersFromConsumerGroup(groupId, new RemoveMembersFromConsumerGroupOptions()).all().get();
|
||||||
throw new IllegalStateException("Consumer group '" + groupId + "' is still active "
|
} else {
|
||||||
|
throw new IllegalStateException("Consumer group '" + groupId + "' is still active "
|
||||||
+ "and has following members: " + members + ". "
|
+ "and has following members: " + members + ". "
|
||||||
+ "Make sure to stop all running application instances before running the reset tool."
|
+ "Make sure to stop all running application instances before running the reset tool."
|
||||||
+ " You can use option '--force' to remove active members from the group.");
|
+ " You can use option '--force' to remove active members from the group.");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (ExecutionException ee) {
|
||||||
|
// If the group ID is not found, this is not an error case
|
||||||
|
if (!(ee.getCause() instanceof GroupIdNotFoundException)) {
|
||||||
|
throw ee;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.kafka.common.KafkaException;
|
||||||
import org.apache.kafka.common.KafkaFuture;
|
import org.apache.kafka.common.KafkaFuture;
|
||||||
import org.apache.kafka.common.Node;
|
import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
||||||
import org.apache.kafka.common.protocol.Errors;
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
import org.apache.kafka.common.requests.ListOffsetsResponse;
|
import org.apache.kafka.common.requests.ListOffsetsResponse;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
@ -569,34 +570,54 @@ public class ConsumerGroupCommand {
|
||||||
switch (state) {
|
switch (state) {
|
||||||
case "Empty":
|
case "Empty":
|
||||||
case "Dead":
|
case "Dead":
|
||||||
Collection<TopicPartition> partitionsToReset = getPartitionsToReset(groupId);
|
result.put(groupId, resetOffsetsForInactiveGroup(groupId));
|
||||||
Map<TopicPartition, OffsetAndMetadata> preparedOffsets = prepareOffsetsToReset(groupId, partitionsToReset);
|
|
||||||
|
|
||||||
// Dry-run is the default behavior if --execute is not specified
|
|
||||||
boolean dryRun = opts.options.has(opts.dryRunOpt) || !opts.options.has(opts.executeOpt);
|
|
||||||
if (!dryRun) {
|
|
||||||
adminClient.alterConsumerGroupOffsets(
|
|
||||||
groupId,
|
|
||||||
preparedOffsets,
|
|
||||||
withTimeoutMs(new AlterConsumerGroupOffsetsOptions())
|
|
||||||
).all().get();
|
|
||||||
}
|
|
||||||
|
|
||||||
result.put(groupId, preparedOffsets);
|
|
||||||
|
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
printError("Assignments can only be reset if the group '" + groupId + "' is inactive, but the current state is " + state + ".", Optional.empty());
|
printError("Assignments can only be reset if the group '" + groupId + "' is inactive, but the current state is " + state + ".", Optional.empty());
|
||||||
result.put(groupId, Collections.emptyMap());
|
result.put(groupId, Collections.emptyMap());
|
||||||
}
|
}
|
||||||
} catch (InterruptedException | ExecutionException e) {
|
} catch (InterruptedException ie) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(ie);
|
||||||
|
} catch (ExecutionException ee) {
|
||||||
|
if (ee.getCause() instanceof GroupIdNotFoundException) {
|
||||||
|
result.put(groupId, resetOffsetsForInactiveGroup(groupId));
|
||||||
|
} else {
|
||||||
|
throw new RuntimeException(ee);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Map<TopicPartition, OffsetAndMetadata> resetOffsetsForInactiveGroup(String groupId) {
|
||||||
|
try {
|
||||||
|
Collection<TopicPartition> partitionsToReset = getPartitionsToReset(groupId);
|
||||||
|
Map<TopicPartition, OffsetAndMetadata> preparedOffsets = prepareOffsetsToReset(groupId, partitionsToReset);
|
||||||
|
|
||||||
|
// Dry-run is the default behavior if --execute is not specified
|
||||||
|
boolean dryRun = opts.options.has(opts.dryRunOpt) || !opts.options.has(opts.executeOpt);
|
||||||
|
if (!dryRun) {
|
||||||
|
adminClient.alterConsumerGroupOffsets(
|
||||||
|
groupId,
|
||||||
|
preparedOffsets,
|
||||||
|
withTimeoutMs(new AlterConsumerGroupOffsetsOptions())
|
||||||
|
).all().get();
|
||||||
|
}
|
||||||
|
|
||||||
|
return preparedOffsets;
|
||||||
|
} catch (InterruptedException ie) {
|
||||||
|
throw new RuntimeException(ie);
|
||||||
|
} catch (ExecutionException ee) {
|
||||||
|
Throwable cause = ee.getCause();
|
||||||
|
if (cause instanceof KafkaException) {
|
||||||
|
throw (KafkaException) cause;
|
||||||
|
} else {
|
||||||
|
throw new RuntimeException(cause);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
Entry<Errors, Map<TopicPartition, Throwable>> deleteOffsets(String groupId, List<String> topics) {
|
Entry<Errors, Map<TopicPartition, Throwable>> deleteOffsets(String groupId, List<String> topics) {
|
||||||
Map<TopicPartition, Throwable> partitionLevelResult = new HashMap<>();
|
Map<TopicPartition, Throwable> partitionLevelResult = new HashMap<>();
|
||||||
Set<String> topicWithPartitions = new HashSet<>();
|
Set<String> topicWithPartitions = new HashSet<>();
|
||||||
|
@ -702,7 +723,7 @@ public class ConsumerGroupCommand {
|
||||||
System.out.printf(format,
|
System.out.printf(format,
|
||||||
tp.topic(),
|
tp.topic(),
|
||||||
tp.partition() >= 0 ? tp.partition() : "Not Provided",
|
tp.partition() >= 0 ? tp.partition() : "Not Provided",
|
||||||
error != null ? "Error: :" + error.getMessage() : "Successful"
|
error != null ? "Error: " + error.getMessage() : "Successful"
|
||||||
);
|
);
|
||||||
});
|
});
|
||||||
System.out.println();
|
System.out.println();
|
||||||
|
@ -1231,8 +1252,10 @@ public class ConsumerGroupCommand {
|
||||||
try {
|
try {
|
||||||
f.get();
|
f.get();
|
||||||
success.put(g, null);
|
success.put(g, null);
|
||||||
} catch (ExecutionException | InterruptedException e) {
|
} catch (InterruptedException ie) {
|
||||||
failed.put(g, e);
|
failed.put(g, ie);
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
failed.put(g, e.getCause());
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
|
|
@ -257,7 +257,7 @@ public class ConsumerGroupCommandOptions extends CommandDefaultOptions {
|
||||||
CommandLineUtils.printUsageAndExit(parser, "Option " + resetOffsetsOpt + " only accepts one of " + executeOpt + " and " + dryRunOpt);
|
CommandLineUtils.printUsageAndExit(parser, "Option " + resetOffsetsOpt + " only accepts one of " + executeOpt + " and " + dryRunOpt);
|
||||||
|
|
||||||
if (!options.has(dryRunOpt) && !options.has(executeOpt)) {
|
if (!options.has(dryRunOpt) && !options.has(executeOpt)) {
|
||||||
System.err.println("WARN: No action will be performed as the --execute option is missing." +
|
System.err.println("WARN: No action will be performed as the --execute option is missing. " +
|
||||||
"In a future major release, the default behavior of this command will be to prompt the user before " +
|
"In a future major release, the default behavior of this command will be to prompt the user before " +
|
||||||
"executing the reset rather than doing a dry run. You should add the --dry-run option explicitly " +
|
"executing the reset rather than doing a dry run. You should add the --dry-run option explicitly " +
|
||||||
"if you are scripting this command and want to keep the current default behavior without prompting.");
|
"if you are scripting this command and want to keep the current default behavior without prompting.");
|
||||||
|
|
|
@ -19,16 +19,20 @@ package org.apache.kafka.tools.consumer.group;
|
||||||
import kafka.api.AbstractAuthorizerIntegrationTest;
|
import kafka.api.AbstractAuthorizerIntegrationTest;
|
||||||
|
|
||||||
import org.apache.kafka.common.acl.AccessControlEntry;
|
import org.apache.kafka.common.acl.AccessControlEntry;
|
||||||
|
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
||||||
|
|
||||||
import org.junit.jupiter.params.ParameterizedTest;
|
import org.junit.jupiter.params.ParameterizedTest;
|
||||||
import org.junit.jupiter.params.provider.ValueSource;
|
import org.junit.jupiter.params.provider.ValueSource;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
|
||||||
import scala.jdk.javaapi.CollectionConverters;
|
import scala.jdk.javaapi.CollectionConverters;
|
||||||
|
|
||||||
import static org.apache.kafka.common.acl.AclOperation.DESCRIBE;
|
import static org.apache.kafka.common.acl.AclOperation.DESCRIBE;
|
||||||
import static org.apache.kafka.common.acl.AclPermissionType.ALLOW;
|
import static org.apache.kafka.common.acl.AclPermissionType.ALLOW;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||||
|
import static org.junit.jupiter.api.Assertions.fail;
|
||||||
|
|
||||||
public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
|
@ -38,8 +42,12 @@ public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest
|
||||||
|
|
||||||
String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()};
|
String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()};
|
||||||
ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(cgcArgs);
|
ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(cgcArgs);
|
||||||
ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Collections.emptyMap());
|
try (ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Collections.emptyMap())) {
|
||||||
consumerGroupService.describeGroups();
|
consumerGroupService.describeGroups();
|
||||||
consumerGroupService.close();
|
fail("Non-existent group should throw an exception");
|
||||||
|
} catch (ExecutionException e) {
|
||||||
|
assertInstanceOf(GroupIdNotFoundException.class, e.getCause(),
|
||||||
|
"Non-existent group should throw GroupIdNotFoundException");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -100,7 +100,7 @@ public class DeleteConsumerGroupsTest {
|
||||||
assertEquals(1, result.size());
|
assertEquals(1, result.size());
|
||||||
assertNotNull(result.get(missingGroupId));
|
assertNotNull(result.get(missingGroupId));
|
||||||
assertInstanceOf(GroupIdNotFoundException.class,
|
assertInstanceOf(GroupIdNotFoundException.class,
|
||||||
result.get(missingGroupId).getCause(),
|
result.get(missingGroupId),
|
||||||
"The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not detected while deleting consumer group");
|
"The expected error (" + Errors.GROUP_ID_NOT_FOUND + ") was not detected while deleting consumer group");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -132,7 +132,7 @@ public class DeleteConsumerGroupsTest {
|
||||||
assertEquals(1, result.size());
|
assertEquals(1, result.size());
|
||||||
assertNotNull(result.get(groupId));
|
assertNotNull(result.get(groupId));
|
||||||
assertInstanceOf(GroupNotEmptyException.class,
|
assertInstanceOf(GroupNotEmptyException.class,
|
||||||
result.get(groupId).getCause(),
|
result.get(groupId),
|
||||||
"The expected error (" + Errors.NON_EMPTY_GROUP + ") was not detected while deleting consumer group. Result was:(" + result + ")");
|
"The expected error (" + Errors.NON_EMPTY_GROUP + ") was not detected while deleting consumer group. Result was:(" + result + ")");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.kafka.clients.consumer.RangeAssignor;
|
||||||
import org.apache.kafka.clients.consumer.RoundRobinAssignor;
|
import org.apache.kafka.clients.consumer.RoundRobinAssignor;
|
||||||
import org.apache.kafka.common.GroupState;
|
import org.apache.kafka.common.GroupState;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
||||||
import org.apache.kafka.common.errors.TimeoutException;
|
import org.apache.kafka.common.errors.TimeoutException;
|
||||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
import org.apache.kafka.common.serialization.StringDeserializer;
|
||||||
import org.apache.kafka.common.test.api.ClusterConfig;
|
import org.apache.kafka.common.test.api.ClusterConfig;
|
||||||
|
@ -68,6 +69,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
import static org.junit.jupiter.api.Assertions.fail;
|
||||||
|
|
||||||
@ExtendWith(value = ClusterTestExtensions.class)
|
@ExtendWith(value = ClusterTestExtensions.class)
|
||||||
public class DescribeConsumerGroupTest {
|
public class DescribeConsumerGroupTest {
|
||||||
|
@ -92,9 +94,13 @@ public class DescribeConsumerGroupTest {
|
||||||
List<String> cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup));
|
List<String> cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup));
|
||||||
cgcArgs.addAll(describeType);
|
cgcArgs.addAll(describeType);
|
||||||
try (ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0]))) {
|
try (ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0]))) {
|
||||||
String output = ToolsTestUtils.grabConsoleOutput(describeGroups(service));
|
service.describeGroups();
|
||||||
assertTrue(output.contains("Consumer group '" + missingGroup + "' does not exist."),
|
fail("Expected error was not detected for describe option '" + String.join(" ", describeType) + "'");
|
||||||
"Expected error was not detected for describe option '" + String.join(" ", describeType) + "'");
|
} catch (ExecutionException ee) {
|
||||||
|
assertInstanceOf(GroupIdNotFoundException.class, ee.getCause());
|
||||||
|
assertEquals("Group " + missingGroup + " not found.", ee.getCause().getMessage());
|
||||||
|
} catch (Exception e) {
|
||||||
|
fail("Expected error was not detected for describe option '" + String.join(" ", describeType) + "'");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -113,9 +119,11 @@ public class DescribeConsumerGroupTest {
|
||||||
// note the group to be queried is a different (non-existing) group
|
// note the group to be queried is a different (non-existing) group
|
||||||
ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup})
|
ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup})
|
||||||
) {
|
) {
|
||||||
Entry<Optional<GroupState>, Optional<Collection<PartitionAssignmentState>>> res = service.collectGroupOffsets(missingGroup);
|
service.collectGroupOffsets(missingGroup);
|
||||||
assertTrue(res.getKey().map(s -> s.equals(GroupState.DEAD)).orElse(false) && res.getValue().map(Collection::isEmpty).orElse(false),
|
fail("Expected the group '" + missingGroup + "' to throw GroupIdNotFoundException");
|
||||||
"Expected the state to be 'Dead', with no members in the group '" + missingGroup + "'.");
|
} catch (ExecutionException ee) {
|
||||||
|
assertInstanceOf(GroupIdNotFoundException.class, ee.getCause(),
|
||||||
|
"Expected the group '" + missingGroup + "' to throw GroupIdNotFoundException");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -132,13 +140,11 @@ public class DescribeConsumerGroupTest {
|
||||||
// note the group to be queried is a different (non-existing) group
|
// note the group to be queried is a different (non-existing) group
|
||||||
ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup})
|
ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup})
|
||||||
) {
|
) {
|
||||||
Entry<Optional<GroupState>, Optional<Collection<MemberAssignmentState>>> res = service.collectGroupMembers(missingGroup, false);
|
service.collectGroupMembers(missingGroup, false);
|
||||||
assertTrue(res.getKey().map(s -> s.equals(GroupState.DEAD)).orElse(false) && res.getValue().map(Collection::isEmpty).orElse(false),
|
fail("Expected the group '" + missingGroup + "' to throw GroupIdNotFoundException");
|
||||||
"Expected the state to be 'Dead', with no members in the group '" + missingGroup + "'.");
|
} catch (ExecutionException ee) {
|
||||||
|
assertInstanceOf(GroupIdNotFoundException.class, ee.getCause(),
|
||||||
Entry<Optional<GroupState>, Optional<Collection<MemberAssignmentState>>> res2 = service.collectGroupMembers(missingGroup, true);
|
"Expected the group '" + missingGroup + "' to throw GroupIdNotFoundException");
|
||||||
assertTrue(res2.getKey().map(s -> s.equals(GroupState.DEAD)).orElse(false) && res2.getValue().map(Collection::isEmpty).orElse(false),
|
|
||||||
"Expected the state to be 'Dead', with no members in the group '" + missingGroup + "' (verbose option).");
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -155,11 +161,11 @@ public class DescribeConsumerGroupTest {
|
||||||
// note the group to be queried is a different (non-existing) group
|
// note the group to be queried is a different (non-existing) group
|
||||||
ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup})
|
ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup})
|
||||||
) {
|
) {
|
||||||
GroupInformation state = service.collectGroupState(missingGroup);
|
service.collectGroupState(missingGroup);
|
||||||
assertTrue(Objects.equals(state.groupState, GroupState.DEAD) && state.numMembers == 0 &&
|
fail("Expected the group '" + missingGroup + "' to throw GroupIdNotFoundException");
|
||||||
state.coordinator != null && clusterInstance.brokerIds().contains(state.coordinator.id()),
|
} catch (ExecutionException ee) {
|
||||||
"Expected the state to be 'Dead', with no members in the group '" + missingGroup + "'."
|
assertInstanceOf(GroupIdNotFoundException.class, ee.getCause(),
|
||||||
);
|
"Expected the group '" + missingGroup + "' to throw GroupIdNotFoundException");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -136,10 +136,6 @@ public class ResetConsumerGroupOffsetTest {
|
||||||
String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--to-current", "--execute");
|
String[] args = buildArgsForGroup(cluster, group, "--all-topics", "--to-current", "--execute");
|
||||||
|
|
||||||
try (ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) {
|
try (ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) {
|
||||||
// Make sure we got a coordinator
|
|
||||||
TestUtils.waitForCondition(
|
|
||||||
() -> "localhost".equals(service.collectGroupState(group).coordinator.host()),
|
|
||||||
"Can't find a coordinator");
|
|
||||||
Map<TopicPartition, OffsetAndMetadata> resetOffsets = service.resetOffsets().get(group);
|
Map<TopicPartition, OffsetAndMetadata> resetOffsets = service.resetOffsets().get(group);
|
||||||
assertTrue(resetOffsets.isEmpty());
|
assertTrue(resetOffsets.isEmpty());
|
||||||
assertTrue(committedOffsets(cluster, topic, group).isEmpty());
|
assertTrue(committedOffsets(cluster, topic, group).isEmpty());
|
||||||
|
|
Loading…
Reference in New Issue