|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | +package org.apache.kafka.clients.admin; |
| 18 | + |
| 19 | +import org.apache.kafka.common.KafkaFuture; |
| 20 | +import org.apache.kafka.common.annotation.InterfaceStability; |
| 21 | +import org.apache.kafka.common.internals.KafkaFutureImpl; |
| 22 | + |
| 23 | +import java.util.ArrayList; |
| 24 | +import java.util.Collection; |
| 25 | +import java.util.HashMap; |
| 26 | +import java.util.HashSet; |
| 27 | +import java.util.List; |
| 28 | +import java.util.Map; |
| 29 | +import java.util.Set; |
| 30 | + |
| 31 | +/** |
| 32 | + * The result of the {@link Admin#listTransactions()} call. |
| 33 | + * <p> |
| 34 | + * The API of this class is evolving, see {@link Admin} for details. |
| 35 | + */ |
| 36 | +@InterfaceStability.Evolving |
| 37 | +public class ListTransactionsResult { |
| 38 | + private final KafkaFutureImpl<Map<Integer, KafkaFutureImpl<Collection<TransactionListing>>>> future; |
| 39 | + |
| 40 | + ListTransactionsResult(KafkaFutureImpl<Map<Integer, KafkaFutureImpl<Collection<TransactionListing>>>> future) { |
| 41 | + this.future = future; |
| 42 | + } |
| 43 | + |
| 44 | + /** |
| 45 | + * Get all transaction listings. If any of the underlying requests fail, then the future |
| 46 | + * returned from this method will also fail with the first encountered error. |
| 47 | + * |
| 48 | + * @return A future containing the collection of transaction listings. The future completes |
| 49 | + * when all transaction listings are available and fails after any non-retriable error. |
| 50 | + */ |
| 51 | + public KafkaFuture<Collection<TransactionListing>> all() { |
| 52 | + return allByBrokerId().thenApply(map -> { |
| 53 | + List<TransactionListing> allListings = new ArrayList<>(); |
| 54 | + for (Collection<TransactionListing> listings : map.values()) { |
| 55 | + allListings.addAll(listings); |
| 56 | + } |
| 57 | + return allListings; |
| 58 | + }); |
| 59 | + } |
| 60 | + |
| 61 | + /** |
| 62 | + * Get a future which returns a map containing the underlying listing future for each broker |
| 63 | + * in the cluster. This is useful, for example, if a partial listing of transactions is |
| 64 | + * sufficient, or if you want more granular error details. |
| 65 | + * |
| 66 | + * @return A future containing a map of futures by broker which complete individually when |
| 67 | + * their respective transaction listings are available. The top-level future returned |
| 68 | + * from this method may fail if the admin client is unable to lookup the available |
| 69 | + * brokers in the cluster. |
| 70 | + */ |
| 71 | + public KafkaFuture<Map<Integer, KafkaFuture<Collection<TransactionListing>>>> byBrokerId() { |
| 72 | + KafkaFutureImpl<Map<Integer, KafkaFuture<Collection<TransactionListing>>>> result = new KafkaFutureImpl<>(); |
| 73 | + future.whenComplete((brokerFutures, exception) -> { |
| 74 | + if (brokerFutures != null) { |
| 75 | + Map<Integer, KafkaFuture<Collection<TransactionListing>>> brokerFuturesCopy = |
| 76 | + new HashMap<>(brokerFutures.size()); |
| 77 | + brokerFuturesCopy.putAll(brokerFutures); |
| 78 | + result.complete(brokerFuturesCopy); |
| 79 | + } else { |
| 80 | + result.completeExceptionally(exception); |
| 81 | + } |
| 82 | + }); |
| 83 | + return result; |
| 84 | + } |
| 85 | + |
| 86 | + /** |
| 87 | + * Get all transaction listings in a map which is keyed by the ID of respective broker |
| 88 | + * that is currently managing them. If any of the underlying requests fail, then the future |
| 89 | + * returned from this method will also fail with the first encountered error. |
| 90 | + * |
| 91 | + * @return A future containing a map from the broker ID to the transactions hosted by that |
| 92 | + * broker respectively. This future completes when all transaction listings are |
| 93 | + * available and fails after any non-retriable error. |
| 94 | + */ |
| 95 | + public KafkaFuture<Map<Integer, Collection<TransactionListing>>> allByBrokerId() { |
| 96 | + KafkaFutureImpl<Map<Integer, Collection<TransactionListing>>> allFuture = new KafkaFutureImpl<>(); |
| 97 | + Map<Integer, Collection<TransactionListing>> allListingsMap = new HashMap<>(); |
| 98 | + |
| 99 | + future.whenComplete((map, topLevelException) -> { |
| 100 | + if (topLevelException != null) { |
| 101 | + allFuture.completeExceptionally(topLevelException); |
| 102 | + return; |
| 103 | + } |
| 104 | + |
| 105 | + Set<Integer> remainingResponses = new HashSet<>(map.keySet()); |
| 106 | + map.forEach((brokerId, future) -> { |
| 107 | + future.whenComplete((listings, brokerException) -> { |
| 108 | + if (brokerException != null) { |
| 109 | + allFuture.completeExceptionally(brokerException); |
| 110 | + } else if (!allFuture.isDone()) { |
| 111 | + allListingsMap.put(brokerId, listings); |
| 112 | + remainingResponses.remove(brokerId); |
| 113 | + |
| 114 | + if (remainingResponses.isEmpty()) { |
| 115 | + allFuture.complete(allListingsMap); |
| 116 | + } |
| 117 | + } |
| 118 | + }); |
| 119 | + }); |
| 120 | + }); |
| 121 | + |
| 122 | + return allFuture; |
| 123 | + } |
| 124 | + |
| 125 | +} |
0 commit comments