Browse Source
This PR adds supporting features for static membership. It could batch remove consumers from the group with provided group.instance.id list. Reviewers: Guozhang Wang <wangguoz@gmail.com>pull/7317/head
Boyang Chen
5 years ago
committed by
Guozhang Wang
13 changed files with 711 additions and 23 deletions
@ -0,0 +1,49 @@ |
|||||||
|
/* |
||||||
|
* 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.annotation.InterfaceStability; |
||||||
|
|
||||||
|
import java.util.concurrent.ExecutionException; |
||||||
|
|
||||||
|
/** |
||||||
|
* The result of the {@link KafkaAdminClient#removeMemberFromConsumerGroup(String, RemoveMemberFromConsumerGroupOptions)} call. |
||||||
|
* |
||||||
|
* The API of this class is evolving, see {@link AdminClient} for details. |
||||||
|
*/ |
||||||
|
@InterfaceStability.Evolving |
||||||
|
public class MembershipChangeResult { |
||||||
|
|
||||||
|
private KafkaFuture<RemoveMemberFromGroupResult> future; |
||||||
|
|
||||||
|
MembershipChangeResult(KafkaFuture<RemoveMemberFromGroupResult> future) { |
||||||
|
this.future = future; |
||||||
|
} |
||||||
|
|
||||||
|
/** |
||||||
|
* Return a future which contains the member removal results. |
||||||
|
*/ |
||||||
|
public RemoveMemberFromGroupResult all() throws ExecutionException, InterruptedException { |
||||||
|
return future.get(); |
||||||
|
} |
||||||
|
|
||||||
|
// Visible for testing
|
||||||
|
public KafkaFuture<RemoveMemberFromGroupResult> future() { |
||||||
|
return future; |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,50 @@ |
|||||||
|
/* |
||||||
|
* 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; |
||||||
|
} |
||||||
|
} |
||||||
|
|
@ -0,0 +1,85 @@ |
|||||||
|
/* |
||||||
|
* 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,50 @@ |
|||||||
|
/* |
||||||
|
* 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"); |
||||||
|
} |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,38 @@ |
|||||||
|
/* |
||||||
|
* 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.junit.Test; |
||||||
|
|
||||||
|
import java.util.Collections; |
||||||
|
import java.util.List; |
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals; |
||||||
|
|
||||||
|
public class RemoveMemberFromConsumerGroupOptionsTest { |
||||||
|
|
||||||
|
@Test |
||||||
|
public void testConstructor() { |
||||||
|
List<String> groupInstanceIds = Collections.singletonList("instance-1"); |
||||||
|
|
||||||
|
RemoveMemberFromConsumerGroupOptions options = new RemoveMemberFromConsumerGroupOptions(groupInstanceIds); |
||||||
|
|
||||||
|
assertEquals(Collections.singletonList( |
||||||
|
new MemberIdentity().setGroupInstanceId("instance-1")), options.getMembers()); |
||||||
|
} |
||||||
|
} |
@ -0,0 +1,154 @@ |
|||||||
|
/* |
||||||
|
* 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"); |
||||||
|
} |
||||||
|
} |
||||||
|
} |
||||||
|
} |
Loading…
Reference in new issue