-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-10000: Add producer fencing API to admin client (KIP-618) #11777
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
d4d6120
KAFKA-10000: Add producer fencing API to admin client
C0urante 0d0dbbf
Fix Log4j class, Javadocs, and whitespace
C0urante 74d589c
Remove unnecessary error-handling cases from FenceProducersHandler, a…
C0urante 6228d2b
KAFKA-10000: Increase testing coverage for KafkaAdminClient::fencePro…
C0urante File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
38 changes: 38 additions & 0 deletions
38
clients/src/main/java/org/apache/kafka/clients/admin/FenceProducersOptions.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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.annotation.InterfaceStability; | ||
|
|
||
| import java.util.Collection; | ||
|
|
||
| /** | ||
| * Options for {@link Admin#fenceProducers(Collection, FenceProducersOptions)} | ||
| * | ||
| * The API of this class is evolving. See {@link Admin} for details. | ||
| */ | ||
| @InterfaceStability.Evolving | ||
| public class FenceProducersOptions extends AbstractOptions<FenceProducersOptions> { | ||
|
|
||
| @Override | ||
| public String toString() { | ||
| return "FenceProducersOptions{" + | ||
| "timeoutMs=" + timeoutMs + | ||
| '}'; | ||
| } | ||
| } |
84 changes: 84 additions & 0 deletions
84
clients/src/main/java/org/apache/kafka/clients/admin/FenceProducersResult.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,84 @@ | ||
| /* | ||
| * 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.clients.admin.internals.CoordinatorKey; | ||
| import org.apache.kafka.common.KafkaFuture; | ||
| import org.apache.kafka.common.annotation.InterfaceStability; | ||
| import org.apache.kafka.common.utils.ProducerIdAndEpoch; | ||
|
|
||
| import java.util.Collection; | ||
| import java.util.Map; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| /** | ||
| * The result of the {@link Admin#fenceProducers(Collection)} call. | ||
| * | ||
| * The API of this class is evolving, see {@link Admin} for details. | ||
| */ | ||
| @InterfaceStability.Evolving | ||
| public class FenceProducersResult { | ||
|
|
||
| private final Map<CoordinatorKey, KafkaFuture<ProducerIdAndEpoch>> futures; | ||
|
|
||
| FenceProducersResult(Map<CoordinatorKey, KafkaFuture<ProducerIdAndEpoch>> futures) { | ||
| this.futures = futures; | ||
| } | ||
|
|
||
| /** | ||
| * Return a map from transactional ID to futures which can be used to check the status of | ||
| * individual fencings. | ||
| */ | ||
| public Map<String, KafkaFuture<Void>> fencedProducers() { | ||
| return futures.entrySet().stream().collect(Collectors.toMap( | ||
| e -> e.getKey().idValue, | ||
| e -> e.getValue().thenApply(p -> null) | ||
| )); | ||
| } | ||
|
|
||
| /** | ||
| * Returns a future that provides the producer ID generated while initializing the given transaction when the request completes. | ||
| */ | ||
| public KafkaFuture<Long> producerId(String transactionalId) { | ||
| return findAndApply(transactionalId, p -> p.producerId); | ||
| } | ||
|
|
||
| /** | ||
| * Returns a future that provides the epoch ID generated while initializing the given transaction when the request completes. | ||
| */ | ||
| public KafkaFuture<Short> epochId(String transactionalId) { | ||
| return findAndApply(transactionalId, p -> p.epoch); | ||
| } | ||
|
|
||
| /** | ||
| * Return a future which succeeds only if all the producer fencings succeed. | ||
| */ | ||
| public KafkaFuture<Void> all() { | ||
| return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])); | ||
| } | ||
|
|
||
| private <T> KafkaFuture<T> findAndApply(String transactionalId, KafkaFuture.BaseFunction<ProducerIdAndEpoch, T> followup) { | ||
| CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId); | ||
| KafkaFuture<ProducerIdAndEpoch> future = futures.get(key); | ||
| if (future == null) { | ||
| throw new IllegalArgumentException("TransactionalId " + | ||
| "`" + transactionalId + "` was not included in the request"); | ||
| } | ||
| return future.thenApply(followup); | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.