Browse Source
This PR fixes the inconsistency involved in the `removeMembersFromGroup` admin API calls: 1. Fail the `all()` request when there is sub level error (either partition or member) 2. Change getMembers() to members() 3. Hide the actual Errors from user 4. Do not expose generated MemberIdentity type 5. Use more consistent naming for Options and Result types Reviewers: Guozhang Wang <wangguoz@gmail.com>, David Jacot <djacot@confluent.io>, Jason Gustafson <jason@confluent.io>pull/7585/head
Boyang Chen
5 years ago
committed by
Jason Gustafson
20 changed files with 634 additions and 576 deletions
@ -0,0 +1,58 @@
@@ -0,0 +1,58 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.clients.admin; |
||||
|
||||
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; |
||||
import org.apache.kafka.common.requests.JoinGroupRequest; |
||||
|
||||
import java.util.Objects; |
||||
|
||||
/** |
||||
* A struct containing information about the member to be removed. |
||||
*/ |
||||
public class MemberToRemove { |
||||
private final String groupInstanceId; |
||||
|
||||
public MemberToRemove(String groupInstanceId) { |
||||
this.groupInstanceId = groupInstanceId; |
||||
} |
||||
|
||||
@Override |
||||
public boolean equals(Object o) { |
||||
if (o instanceof MemberToRemove) { |
||||
MemberToRemove otherMember = (MemberToRemove) o; |
||||
return this.groupInstanceId.equals(otherMember.groupInstanceId); |
||||
} else { |
||||
return false; |
||||
} |
||||
} |
||||
|
||||
@Override |
||||
public int hashCode() { |
||||
return Objects.hash(groupInstanceId); |
||||
} |
||||
|
||||
MemberIdentity toMemberIdentity() { |
||||
return new MemberIdentity() |
||||
.setGroupInstanceId(groupInstanceId) |
||||
.setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID); |
||||
} |
||||
|
||||
public String groupInstanceId() { |
||||
return groupInstanceId; |
||||
} |
||||
} |
@ -1,50 +0,0 @@
@@ -1,50 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.clients.admin; |
||||
|
||||
import org.apache.kafka.common.annotation.InterfaceStability; |
||||
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; |
||||
import org.apache.kafka.common.requests.JoinGroupRequest; |
||||
|
||||
import java.util.Collection; |
||||
import java.util.List; |
||||
import java.util.stream.Collectors; |
||||
|
||||
/** |
||||
* Options for {@link AdminClient#removeMemberFromConsumerGroup(String, RemoveMemberFromConsumerGroupOptions)}. |
||||
* It carries the members to be removed from the consumer group. |
||||
* |
||||
* The API of this class is evolving, see {@link AdminClient} for details. |
||||
*/ |
||||
@InterfaceStability.Evolving |
||||
public class RemoveMemberFromConsumerGroupOptions extends AbstractOptions<RemoveMemberFromConsumerGroupOptions> { |
||||
|
||||
private List<MemberIdentity> members; |
||||
|
||||
public RemoveMemberFromConsumerGroupOptions(Collection<String> groupInstanceIds) { |
||||
members = groupInstanceIds.stream().map( |
||||
instanceId -> new MemberIdentity() |
||||
.setGroupInstanceId(instanceId) |
||||
.setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID) |
||||
).collect(Collectors.toList()); |
||||
} |
||||
|
||||
public List<MemberIdentity> getMembers() { |
||||
return members; |
||||
} |
||||
} |
||||
|
@ -1,85 +0,0 @@
@@ -1,85 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.clients.admin; |
||||
|
||||
import org.apache.kafka.common.KafkaFuture; |
||||
import org.apache.kafka.common.internals.KafkaFutureImpl; |
||||
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; |
||||
import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; |
||||
import org.apache.kafka.common.protocol.Errors; |
||||
import org.apache.kafka.common.requests.LeaveGroupResponse; |
||||
|
||||
import java.util.HashMap; |
||||
import java.util.List; |
||||
import java.util.Map; |
||||
|
||||
/** |
||||
* Result of a batch member removal operation. |
||||
*/ |
||||
public class RemoveMemberFromGroupResult { |
||||
|
||||
private final Errors topLevelError; |
||||
private final Map<MemberIdentity, KafkaFuture<Void>> memberFutures; |
||||
private boolean hasError = false; |
||||
|
||||
RemoveMemberFromGroupResult(LeaveGroupResponse response, |
||||
List<MemberIdentity> membersToRemove) { |
||||
this.topLevelError = response.topLevelError(); |
||||
this.memberFutures = new HashMap<>(membersToRemove.size()); |
||||
|
||||
if (this.topLevelError != Errors.NONE) { |
||||
// If the populated error is a top-level error, fail every member's future.
|
||||
for (MemberIdentity memberIdentity : membersToRemove) { |
||||
KafkaFutureImpl<Void> future = new KafkaFutureImpl<>(); |
||||
future.completeExceptionally(topLevelError.exception()); |
||||
memberFutures.put(memberIdentity, future); |
||||
} |
||||
hasError = true; |
||||
} else { |
||||
for (MemberResponse memberResponse : response.memberResponses()) { |
||||
KafkaFutureImpl<Void> future = new KafkaFutureImpl<>(); |
||||
Errors memberError = Errors.forCode(memberResponse.errorCode()); |
||||
if (memberError != Errors.NONE) { |
||||
future.completeExceptionally(memberError.exception()); |
||||
hasError = true; |
||||
} else { |
||||
future.complete(null); |
||||
} |
||||
memberFutures.put(new MemberIdentity() |
||||
.setMemberId(memberResponse.memberId()) |
||||
.setGroupInstanceId(memberResponse.groupInstanceId()), future); |
||||
} |
||||
} |
||||
} |
||||
|
||||
public Errors topLevelError() { |
||||
return topLevelError; |
||||
} |
||||
|
||||
public boolean hasError() { |
||||
return hasError; |
||||
} |
||||
|
||||
/** |
||||
* Futures of members with corresponding errors when they leave the group. |
||||
* |
||||
* @return list of members who failed to be removed |
||||
*/ |
||||
public Map<MemberIdentity, KafkaFuture<Void>> memberFutures() { |
||||
return memberFutures; |
||||
} |
||||
} |
@ -0,0 +1,96 @@
@@ -0,0 +1,96 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.clients.admin; |
||||
|
||||
import org.apache.kafka.common.KafkaFuture; |
||||
import org.apache.kafka.common.internals.KafkaFutureImpl; |
||||
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; |
||||
import org.apache.kafka.common.protocol.Errors; |
||||
|
||||
import java.util.Map; |
||||
import java.util.Set; |
||||
|
||||
/** |
||||
* The result of the {@link Admin#removeMembersFromConsumerGroup(String, RemoveMembersFromConsumerGroupOptions)} call. |
||||
* |
||||
* The API of this class is evolving, see {@link Admin} for details. |
||||
*/ |
||||
public class RemoveMembersFromConsumerGroupResult { |
||||
|
||||
private final KafkaFuture<Map<MemberIdentity, Errors>> future; |
||||
private final Set<MemberToRemove> memberInfos; |
||||
|
||||
RemoveMembersFromConsumerGroupResult(KafkaFuture<Map<MemberIdentity, Errors>> future, |
||||
Set<MemberToRemove> memberInfos) { |
||||
this.future = future; |
||||
this.memberInfos = memberInfos; |
||||
} |
||||
|
||||
/** |
||||
* Returns a future which indicates whether the request was 100% success, i.e. no |
||||
* either top level or member level error. |
||||
* If not, the first member error shall be returned. |
||||
*/ |
||||
public KafkaFuture<Void> all() { |
||||
final KafkaFutureImpl<Void> result = new KafkaFutureImpl<>(); |
||||
this.future.whenComplete((memberErrors, throwable) -> { |
||||
if (throwable != null) { |
||||
result.completeExceptionally(throwable); |
||||
} else { |
||||
for (MemberToRemove memberToRemove : memberInfos) { |
||||
if (maybeCompleteExceptionally(memberErrors, memberToRemove.toMemberIdentity(), result)) { |
||||
return; |
||||
} |
||||
} |
||||
result.complete(null); |
||||
} |
||||
}); |
||||
return result; |
||||
} |
||||
|
||||
/** |
||||
* Returns the selected member future. |
||||
*/ |
||||
public KafkaFuture<Void> memberResult(MemberToRemove member) { |
||||
if (!memberInfos.contains(member)) { |
||||
throw new IllegalArgumentException("Member " + member + " was not included in the original request"); |
||||
} |
||||
|
||||
final KafkaFutureImpl<Void> result = new KafkaFutureImpl<>(); |
||||
this.future.whenComplete((memberErrors, throwable) -> { |
||||
if (throwable != null) { |
||||
result.completeExceptionally(throwable); |
||||
} else if (!maybeCompleteExceptionally(memberErrors, member.toMemberIdentity(), result)) { |
||||
result.complete(null); |
||||
} |
||||
}); |
||||
return result; |
||||
} |
||||
|
||||
private boolean maybeCompleteExceptionally(Map<MemberIdentity, Errors> memberErrors, |
||||
MemberIdentity member, |
||||
KafkaFutureImpl<Void> result) { |
||||
Throwable exception = KafkaAdminClient.getSubLevelError(memberErrors, member, |
||||
"Member \"" + member + "\" was not included in the removal response"); |
||||
if (exception != null) { |
||||
result.completeExceptionally(exception); |
||||
return true; |
||||
} else { |
||||
return false; |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,118 @@
@@ -0,0 +1,118 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.clients.admin; |
||||
|
||||
import org.apache.kafka.common.TopicPartition; |
||||
import org.apache.kafka.common.errors.GroupAuthorizationException; |
||||
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; |
||||
import org.apache.kafka.common.internals.KafkaFutureImpl; |
||||
|
||||
import org.apache.kafka.common.protocol.Errors; |
||||
import org.apache.kafka.test.TestUtils; |
||||
import org.junit.Before; |
||||
import org.junit.Test; |
||||
|
||||
import java.util.HashMap; |
||||
import java.util.HashSet; |
||||
import java.util.Map; |
||||
import java.util.Set; |
||||
import java.util.concurrent.ExecutionException; |
||||
|
||||
import static org.junit.Assert.assertFalse; |
||||
import static org.junit.Assert.assertNull; |
||||
import static org.junit.Assert.assertThrows; |
||||
|
||||
public class DeleteConsumerGroupOffsetsResultTest { |
||||
|
||||
private final String topic = "topic"; |
||||
private final TopicPartition tpZero = new TopicPartition(topic, 0); |
||||
private final TopicPartition tpOne = new TopicPartition(topic, 1); |
||||
private Set<TopicPartition> partitions; |
||||
private Map<TopicPartition, Errors> errorsMap; |
||||
|
||||
private KafkaFutureImpl<Map<TopicPartition, Errors>> partitionFutures; |
||||
|
||||
@Before |
||||
public void setUp() { |
||||
partitionFutures = new KafkaFutureImpl<>(); |
||||
partitions = new HashSet<>(); |
||||
partitions.add(tpZero); |
||||
partitions.add(tpOne); |
||||
|
||||
errorsMap = new HashMap<>(); |
||||
errorsMap.put(tpZero, Errors.NONE); |
||||
errorsMap.put(tpOne, Errors.UNKNOWN_TOPIC_OR_PARTITION); |
||||
} |
||||
|
||||
@Test |
||||
public void testTopLevelErrorConstructor() throws InterruptedException { |
||||
partitionFutures.completeExceptionally(Errors.GROUP_AUTHORIZATION_FAILED.exception()); |
||||
DeleteConsumerGroupOffsetsResult topLevelErrorResult = |
||||
new DeleteConsumerGroupOffsetsResult(partitionFutures, partitions); |
||||
TestUtils.assertFutureError(topLevelErrorResult.all(), GroupAuthorizationException.class); |
||||
} |
||||
|
||||
@Test |
||||
public void testPartitionLevelErrorConstructor() throws ExecutionException, InterruptedException { |
||||
createAndVerifyPartitionLevelErrror(); |
||||
} |
||||
|
||||
@Test |
||||
public void testPartitionMissingInResponseErrorConstructor() throws InterruptedException, ExecutionException { |
||||
errorsMap.remove(tpOne); |
||||
partitionFutures.complete(errorsMap); |
||||
assertFalse(partitionFutures.isCompletedExceptionally()); |
||||
DeleteConsumerGroupOffsetsResult missingPartitionResult = |
||||
new DeleteConsumerGroupOffsetsResult(partitionFutures, partitions); |
||||
|
||||
TestUtils.assertFutureError(missingPartitionResult.all(), IllegalArgumentException.class); |
||||
assertNull(missingPartitionResult.partitionResult(tpZero).get()); |
||||
TestUtils.assertFutureError(missingPartitionResult.partitionResult(tpOne), IllegalArgumentException.class); |
||||
} |
||||
|
||||
@Test |
||||
public void testPartitionMissingInRequestErrorConstructor() throws InterruptedException, ExecutionException { |
||||
DeleteConsumerGroupOffsetsResult partitionLevelErrorResult = createAndVerifyPartitionLevelErrror(); |
||||
assertThrows(IllegalArgumentException.class, () -> partitionLevelErrorResult.partitionResult(new TopicPartition("invalid-topic", 0))); |
||||
} |
||||
|
||||
@Test |
||||
public void testNoErrorConstructor() throws ExecutionException, InterruptedException { |
||||
Map<TopicPartition, Errors> errorsMap = new HashMap<>(); |
||||
errorsMap.put(tpZero, Errors.NONE); |
||||
errorsMap.put(tpOne, Errors.NONE); |
||||
DeleteConsumerGroupOffsetsResult noErrorResult = |
||||
new DeleteConsumerGroupOffsetsResult(partitionFutures, partitions); |
||||
partitionFutures.complete(errorsMap); |
||||
|
||||
assertNull(noErrorResult.all().get()); |
||||
assertNull(noErrorResult.partitionResult(tpZero).get()); |
||||
assertNull(noErrorResult.partitionResult(tpOne).get()); |
||||
} |
||||
|
||||
private DeleteConsumerGroupOffsetsResult createAndVerifyPartitionLevelErrror() throws InterruptedException, ExecutionException { |
||||
partitionFutures.complete(errorsMap); |
||||
assertFalse(partitionFutures.isCompletedExceptionally()); |
||||
DeleteConsumerGroupOffsetsResult partitionLevelErrorResult = |
||||
new DeleteConsumerGroupOffsetsResult(partitionFutures, partitions); |
||||
|
||||
TestUtils.assertFutureError(partitionLevelErrorResult.all(), UnknownTopicOrPartitionException.class); |
||||
assertNull(partitionLevelErrorResult.partitionResult(tpZero).get()); |
||||
TestUtils.assertFutureError(partitionLevelErrorResult.partitionResult(tpOne), UnknownTopicOrPartitionException.class); |
||||
return partitionLevelErrorResult; |
||||
} |
||||
} |
@ -1,50 +0,0 @@
@@ -1,50 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.clients.admin; |
||||
|
||||
import org.apache.kafka.common.internals.KafkaFutureImpl; |
||||
import org.apache.kafka.common.message.LeaveGroupResponseData; |
||||
import org.apache.kafka.common.requests.LeaveGroupResponse; |
||||
import org.junit.Test; |
||||
|
||||
import java.util.Collections; |
||||
import java.util.concurrent.ExecutionException; |
||||
|
||||
import static org.junit.Assert.assertEquals; |
||||
import static org.junit.Assert.fail; |
||||
|
||||
public class MembershipChangeResultTest { |
||||
|
||||
@Test |
||||
public void testConstructor() { |
||||
KafkaFutureImpl<RemoveMemberFromGroupResult> removeMemberFuture = new KafkaFutureImpl<>(); |
||||
|
||||
MembershipChangeResult changeResult = new MembershipChangeResult(removeMemberFuture); |
||||
assertEquals(removeMemberFuture, changeResult.future()); |
||||
RemoveMemberFromGroupResult removeMemberFromGroupResult = new RemoveMemberFromGroupResult( |
||||
new LeaveGroupResponse(new LeaveGroupResponseData()), |
||||
Collections.emptyList() |
||||
); |
||||
|
||||
removeMemberFuture.complete(removeMemberFromGroupResult); |
||||
try { |
||||
assertEquals(removeMemberFromGroupResult, changeResult.all()); |
||||
} catch (ExecutionException | InterruptedException e) { |
||||
fail("Unexpected exception " + e + " when trying to get remove member result"); |
||||
} |
||||
} |
||||
} |
@ -1,154 +0,0 @@
@@ -1,154 +0,0 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.clients.admin; |
||||
|
||||
import org.apache.kafka.common.KafkaFuture; |
||||
import org.apache.kafka.common.errors.FencedInstanceIdException; |
||||
import org.apache.kafka.common.errors.GroupAuthorizationException; |
||||
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; |
||||
import org.apache.kafka.common.message.LeaveGroupResponseData; |
||||
import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; |
||||
import org.apache.kafka.common.protocol.Errors; |
||||
import org.apache.kafka.common.requests.LeaveGroupResponse; |
||||
import org.junit.Before; |
||||
import org.junit.Test; |
||||
|
||||
import java.util.Arrays; |
||||
import java.util.List; |
||||
import java.util.Map; |
||||
import java.util.concurrent.ExecutionException; |
||||
|
||||
import static org.junit.Assert.assertFalse; |
||||
import static org.junit.Assert.assertTrue; |
||||
import static org.junit.Assert.assertEquals; |
||||
import static org.junit.Assert.fail; |
||||
|
||||
public class RemoveMemberFromGroupResultTest { |
||||
|
||||
private String instanceOne = "instance-1"; |
||||
private String instanceTwo = "instance-2"; |
||||
private List<MemberIdentity> membersToRemove; |
||||
|
||||
private List<MemberResponse> memberResponses; |
||||
|
||||
@Before |
||||
public void setUp() { |
||||
membersToRemove = Arrays.asList( |
||||
new MemberIdentity() |
||||
.setGroupInstanceId(instanceOne), |
||||
new MemberIdentity() |
||||
.setGroupInstanceId(instanceTwo) |
||||
); |
||||
|
||||
memberResponses = Arrays.asList( |
||||
new MemberResponse() |
||||
.setGroupInstanceId(instanceOne), |
||||
new MemberResponse() |
||||
.setGroupInstanceId(instanceTwo) |
||||
); |
||||
} |
||||
|
||||
@Test |
||||
public void testTopLevelErrorConstructor() { |
||||
RemoveMemberFromGroupResult topLevelErrorResult = |
||||
new RemoveMemberFromGroupResult(new LeaveGroupResponse( |
||||
new LeaveGroupResponseData() |
||||
.setErrorCode(Errors.GROUP_AUTHORIZATION_FAILED.code()) |
||||
.setMembers(memberResponses)), membersToRemove); |
||||
|
||||
assertTrue(topLevelErrorResult.hasError()); |
||||
assertEquals(Errors.GROUP_AUTHORIZATION_FAILED, topLevelErrorResult.topLevelError()); |
||||
|
||||
Map<MemberIdentity, KafkaFuture<Void>> memberFutures = topLevelErrorResult.memberFutures(); |
||||
assertEquals(2, memberFutures.size()); |
||||
for (Map.Entry<MemberIdentity, KafkaFuture<Void>> entry : memberFutures.entrySet()) { |
||||
KafkaFuture<Void> memberFuture = entry.getValue(); |
||||
assertTrue(memberFuture.isCompletedExceptionally()); |
||||
try { |
||||
memberFuture.get(); |
||||
fail("get() should throw ExecutionException"); |
||||
} catch (ExecutionException | InterruptedException e0) { |
||||
assertTrue(e0.getCause() instanceof GroupAuthorizationException); |
||||
} |
||||
} |
||||
} |
||||
|
||||
@Test |
||||
public void testMemberLevelErrorConstructor() { |
||||
MemberResponse responseOne = new MemberResponse() |
||||
.setGroupInstanceId(instanceOne) |
||||
.setErrorCode(Errors.FENCED_INSTANCE_ID.code()); |
||||
MemberResponse responseTwo = new MemberResponse() |
||||
.setGroupInstanceId(instanceTwo) |
||||
.setErrorCode(Errors.NONE.code()); |
||||
|
||||
RemoveMemberFromGroupResult memberLevelErrorResult = new RemoveMemberFromGroupResult( |
||||
new LeaveGroupResponse(new LeaveGroupResponseData() |
||||
.setMembers(Arrays.asList(responseOne, responseTwo))), |
||||
membersToRemove); |
||||
assertTrue(memberLevelErrorResult.hasError()); |
||||
assertEquals(Errors.NONE, memberLevelErrorResult.topLevelError()); |
||||
|
||||
Map<MemberIdentity, KafkaFuture<Void>> memberFutures = memberLevelErrorResult.memberFutures(); |
||||
assertEquals(2, memberFutures.size()); |
||||
for (Map.Entry<MemberIdentity, KafkaFuture<Void>> entry : memberFutures.entrySet()) { |
||||
KafkaFuture<Void> memberFuture = entry.getValue(); |
||||
if (entry.getKey().groupInstanceId().equals(instanceOne)) { |
||||
assertTrue(memberFuture.isCompletedExceptionally()); |
||||
try { |
||||
memberFuture.get(); |
||||
fail("get() should throw ExecutionException"); |
||||
} catch (ExecutionException | InterruptedException e0) { |
||||
assertTrue(e0.getCause() instanceof FencedInstanceIdException); |
||||
} |
||||
} else { |
||||
assertFalse(memberFuture.isCompletedExceptionally()); |
||||
try { |
||||
memberFuture.get(); |
||||
} catch (ExecutionException | InterruptedException e0) { |
||||
fail("get() shouldn't throw exception"); |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
@Test |
||||
public void testNoErrorConstructor() { |
||||
MemberResponse responseOne = new MemberResponse() |
||||
.setGroupInstanceId(instanceOne) |
||||
.setErrorCode(Errors.NONE.code()); |
||||
MemberResponse responseTwo = new MemberResponse() |
||||
.setGroupInstanceId(instanceTwo) |
||||
.setErrorCode(Errors.NONE.code()); |
||||
// If no error is specified, failed members are not visible.
|
||||
RemoveMemberFromGroupResult noErrorResult = new RemoveMemberFromGroupResult( |
||||
new LeaveGroupResponse(new LeaveGroupResponseData() |
||||
.setMembers(Arrays.asList(responseOne, responseTwo))), |
||||
membersToRemove); |
||||
assertFalse(noErrorResult.hasError()); |
||||
assertEquals(Errors.NONE, noErrorResult.topLevelError()); |
||||
Map<MemberIdentity, KafkaFuture<Void>> memberFutures = noErrorResult.memberFutures(); |
||||
assertEquals(2, memberFutures.size()); |
||||
for (Map.Entry<MemberIdentity, KafkaFuture<Void>> entry : memberFutures.entrySet()) { |
||||
try { |
||||
entry.getValue().get(); |
||||
} catch (ExecutionException | InterruptedException e0) { |
||||
fail("get() shouldn't throw exception"); |
||||
} |
||||
} |
||||
} |
||||
} |
@ -0,0 +1,119 @@
@@ -0,0 +1,119 @@
|
||||
/* |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
package org.apache.kafka.clients.admin; |
||||
|
||||
import org.apache.kafka.common.errors.FencedInstanceIdException; |
||||
import org.apache.kafka.common.errors.GroupAuthorizationException; |
||||
import org.apache.kafka.common.internals.KafkaFutureImpl; |
||||
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity; |
||||
|
||||
import org.apache.kafka.common.protocol.Errors; |
||||
import org.apache.kafka.test.TestUtils; |
||||
import org.junit.Before; |
||||
import org.junit.Test; |
||||
|
||||
import java.util.HashMap; |
||||
import java.util.HashSet; |
||||
import java.util.Map; |
||||
import java.util.Set; |
||||
import java.util.concurrent.ExecutionException; |
||||
|
||||
import static org.junit.Assert.assertFalse; |
||||
import static org.junit.Assert.assertNull; |
||||
import static org.junit.Assert.assertThrows; |
||||
|
||||
public class RemoveMembersFromConsumerGroupResultTest { |
||||
|
||||
private final MemberToRemove instanceOne = new MemberToRemove("instance-1"); |
||||
private final MemberToRemove instanceTwo = new MemberToRemove("instance-2"); |
||||
private Set<MemberToRemove> membersToRemove; |
||||
private Map<MemberIdentity, Errors> errorsMap; |
||||
|
||||
private KafkaFutureImpl<Map<MemberIdentity, Errors>> memberFutures; |
||||
|
||||
@Before |
||||
public void setUp() { |
||||
memberFutures = new KafkaFutureImpl<>(); |
||||
membersToRemove = new HashSet<>(); |
||||
membersToRemove.add(instanceOne); |
||||
membersToRemove.add(instanceTwo); |
||||
|
||||
errorsMap = new HashMap<>(); |
||||
errorsMap.put(instanceOne.toMemberIdentity(), Errors.NONE); |
||||
errorsMap.put(instanceTwo.toMemberIdentity(), Errors.FENCED_INSTANCE_ID); |
||||
} |
||||
|
||||
@Test |
||||
public void testTopLevelErrorConstructor() throws InterruptedException { |
||||
memberFutures.completeExceptionally(Errors.GROUP_AUTHORIZATION_FAILED.exception()); |
||||
RemoveMembersFromConsumerGroupResult topLevelErrorResult = |
||||
new RemoveMembersFromConsumerGroupResult(memberFutures, membersToRemove); |
||||
TestUtils.assertFutureError(topLevelErrorResult.all(), GroupAuthorizationException.class); |
||||
} |
||||
|
||||
@Test |
||||
public void testMemberLevelErrorConstructor() throws InterruptedException, ExecutionException { |
||||
createAndVerifyMemberLevelError(); |
||||
} |
||||
|
||||
@Test |
||||
public void testMemberMissingErrorInRequestConstructor() throws InterruptedException, ExecutionException { |
||||
errorsMap.remove(instanceTwo.toMemberIdentity()); |
||||
memberFutures.complete(errorsMap); |
||||
assertFalse(memberFutures.isCompletedExceptionally()); |
||||
RemoveMembersFromConsumerGroupResult missingMemberResult = |
||||
new RemoveMembersFromConsumerGroupResult(memberFutures, membersToRemove); |
||||
|
||||
TestUtils.assertFutureError(missingMemberResult.all(), IllegalArgumentException.class); |
||||
assertNull(missingMemberResult.memberResult(instanceOne).get()); |
||||
TestUtils.assertFutureError(missingMemberResult.memberResult(instanceTwo), IllegalArgumentException.class); |
||||
} |
||||
|
||||
@Test |
||||
public void testMemberLevelErrorInResponseConstructor() throws InterruptedException, ExecutionException { |
||||
RemoveMembersFromConsumerGroupResult memberLevelErrorResult = createAndVerifyMemberLevelError(); |
||||
assertThrows(IllegalArgumentException.class, () -> memberLevelErrorResult.memberResult( |
||||
new MemberToRemove("invalid-instance-id")) |
||||
); |
||||
} |
||||
|
||||
@Test |
||||
public void testNoErrorConstructor() throws ExecutionException, InterruptedException { |
||||
Map<MemberIdentity, Errors> errorsMap = new HashMap<>(); |
||||
errorsMap.put(instanceOne.toMemberIdentity(), Errors.NONE); |
||||
errorsMap.put(instanceTwo.toMemberIdentity(), Errors.NONE); |
||||
RemoveMembersFromConsumerGroupResult noErrorResult = |
||||
new RemoveMembersFromConsumerGroupResult(memberFutures, membersToRemove); |
||||
memberFutures.complete(errorsMap); |
||||
|
||||
assertNull(noErrorResult.all().get()); |
||||
assertNull(noErrorResult.memberResult(instanceOne).get()); |
||||
assertNull(noErrorResult.memberResult(instanceTwo).get()); |
||||
} |
||||
|
||||
private RemoveMembersFromConsumerGroupResult createAndVerifyMemberLevelError() throws InterruptedException, ExecutionException { |
||||
memberFutures.complete(errorsMap); |
||||
assertFalse(memberFutures.isCompletedExceptionally()); |
||||
RemoveMembersFromConsumerGroupResult memberLevelErrorResult = |
||||
new RemoveMembersFromConsumerGroupResult(memberFutures, membersToRemove); |
||||
|
||||
TestUtils.assertFutureError(memberLevelErrorResult.all(), FencedInstanceIdException.class); |
||||
assertNull(memberLevelErrorResult.memberResult(instanceOne).get()); |
||||
TestUtils.assertFutureError(memberLevelErrorResult.memberResult(instanceTwo), FencedInstanceIdException.class); |
||||
return memberLevelErrorResult; |
||||
} |
||||
} |
Loading…
Reference in new issue