-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KIP-396: Add AlterConsumerGroup/List Offsets to AdminClient #7296
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
5426cf7
522a160
ebdda5d
666da34
4297358
13f1714
bf0876c
3843624
502bdc4
e92459f
f5ac2ee
adb3377
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,28 @@ | ||
| /* | ||
| * 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; | ||
|
|
||
| /** | ||
| * Options for the {@link AdminClient#alterConsumerGroupOffsets(String, Map)} call. | ||
| * | ||
| * The API of this class is evolving, see {@link AdminClient} for details. | ||
| */ | ||
| @InterfaceStability.Evolving | ||
| public class AlterConsumerGroupOffsetsOptions extends AbstractOptions<AlterConsumerGroupOffsetsOptions> { | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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 java.util.List; | ||
| import java.util.Map; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import org.apache.kafka.common.KafkaFuture; | ||
| import org.apache.kafka.common.KafkaFuture.BaseFunction; | ||
| import org.apache.kafka.common.KafkaFuture.BiConsumer; | ||
| import org.apache.kafka.common.TopicPartition; | ||
| import org.apache.kafka.common.annotation.InterfaceStability; | ||
| import org.apache.kafka.common.internals.KafkaFutureImpl; | ||
| import org.apache.kafka.common.protocol.Errors; | ||
|
|
||
| /** | ||
| * The result of the {@link AdminClient#alterConsumerGroupOffsets(String, Map)} call. | ||
| * | ||
| * The API of this class is evolving, see {@link AdminClient} for details. | ||
| */ | ||
| @InterfaceStability.Evolving | ||
| public class AlterConsumerGroupOffsetsResult { | ||
|
|
||
| private final KafkaFuture<Map<TopicPartition, Errors>> future; | ||
|
|
||
| AlterConsumerGroupOffsetsResult(KafkaFuture<Map<TopicPartition, Errors>> future) { | ||
| this.future = future; | ||
| } | ||
|
|
||
| /** | ||
| * Return a future which can be used to check the result for a given partition. | ||
| */ | ||
| public KafkaFuture<Void> partitionResult(final TopicPartition partition) { | ||
| final KafkaFutureImpl<Void> result = new KafkaFutureImpl<>(); | ||
|
|
||
| this.future.whenComplete(new BiConsumer<Map<TopicPartition, Errors>, Throwable>() { | ||
| @Override | ||
| public void accept(final Map<TopicPartition, Errors> topicPartitions, final Throwable throwable) { | ||
| if (throwable != null) { | ||
| result.completeExceptionally(throwable); | ||
| } else if (!topicPartitions.containsKey(partition)) { | ||
| result.completeExceptionally(new IllegalArgumentException( | ||
| "Alter offset for partition \"" + partition + "\" was not attempted")); | ||
| } else { | ||
| final Errors error = topicPartitions.get(partition); | ||
| if (error == Errors.NONE) { | ||
| result.complete(null); | ||
| } else { | ||
| result.completeExceptionally(error.exception()); | ||
| } | ||
| } | ||
|
|
||
| } | ||
| }); | ||
|
|
||
| return result; | ||
| } | ||
|
|
||
| /** | ||
| * Return a future which succeeds if all the alter offsets succeed. | ||
| */ | ||
| public KafkaFuture<Void> all() { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm.. This is a little different from what we have in
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's a fair point but I am not sure what the best one is. The rational behind not looking at individual topic/partitions was that it allows to use To be more concrete, it allows to do the following: DeleteConsumerGroupOffsetsResult result = ...;
try {
// wait for the whole group, only raise when a group level or
// transport level exception affection the whole request occurs
result.all().get()
// inspect individual topic/partition
try {
result.partitionResult(...).get()
} catch (Exception e) {
// handle partition exception
}
} catch (Exception e) {
// handle group level exception
}I think that this facilitates the error handling. What do you think?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's an interesting point. I think the usual semantics of
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just made a pass on all XXXResult classes and I think the API semantics are a bit inconsistency in general: originally I thought we only need the But it seems like for results that only contain a Honestly I think not all
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, unfortunately the admin APIs have such a big surface area it's hard to maintain consistency. I think the original intent is what I described though. |
||
| return this.future.thenApply(new BaseFunction<Map<TopicPartition, Errors>, Void>() { | ||
| @Override | ||
| public Void apply(final Map<TopicPartition, Errors> topicPartitionErrorsMap) { | ||
| List<TopicPartition> partitionsFailed = topicPartitionErrorsMap.entrySet() | ||
| .stream() | ||
| .filter(e -> e.getValue() != Errors.NONE) | ||
| .map(Map.Entry::getKey) | ||
| .collect(Collectors.toList()); | ||
| for (Errors error : topicPartitionErrorsMap.values()) { | ||
| if (error != Errors.NONE) { | ||
| throw error.exception( | ||
| "Failed altering consumer group offsets for the following partitions: " + partitionsFailed); | ||
| } | ||
| } | ||
| return null; | ||
| } | ||
| }); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.