|
117 | 117 | import org.apache.kafka.common.message.DescribeClusterResponseData;
|
118 | 118 | import org.apache.kafka.common.message.DescribeConfigsRequestData;
|
119 | 119 | import org.apache.kafka.common.message.DescribeConfigsResponseData;
|
| 120 | +import org.apache.kafka.common.message.DescribeProducersResponseData; |
120 | 121 | import org.apache.kafka.common.message.DescribeTransactionsResponseData;
|
121 | 122 | import org.apache.kafka.common.message.EndTxnRequestData;
|
122 | 123 | import org.apache.kafka.common.message.EndTxnResponseData;
|
|
162 | 163 | import org.apache.kafka.common.requests.DescribeConfigsRequest;
|
163 | 164 | import org.apache.kafka.common.requests.DescribeConfigsResponse;
|
164 | 165 | import org.apache.kafka.common.requests.DescribeGroupsRequest;
|
| 166 | +import org.apache.kafka.common.requests.DescribeProducersRequest; |
| 167 | +import org.apache.kafka.common.requests.DescribeProducersResponse; |
165 | 168 | import org.apache.kafka.common.requests.DescribeTransactionsRequest;
|
166 | 169 | import org.apache.kafka.common.requests.DescribeTransactionsResponse;
|
167 | 170 | import org.apache.kafka.common.requests.EndTxnRequest;
|
@@ -2020,6 +2023,99 @@ protected void handleDescribeGroupRequest(KafkaHeaderAndRequest describeGroup,
|
2020 | 2023 | ));
|
2021 | 2024 | }
|
2022 | 2025 |
|
| 2026 | + @Override |
| 2027 | + protected void handleDescribeProducersRequest(KafkaHeaderAndRequest describeGroup, |
| 2028 | + CompletableFuture<AbstractResponse> responseFuture) { |
| 2029 | + // https://github.com/apache/kafka/blob/79c19da68d6a93a729a07dfdd37f238246653a46/ |
| 2030 | + // core/src/main/scala/kafka/server/KafkaApis.scala#L3397 |
| 2031 | + checkArgument(describeGroup.getRequest() instanceof DescribeProducersRequest); |
| 2032 | + DescribeProducersRequest request = (DescribeProducersRequest) describeGroup.getRequest(); |
| 2033 | + Map<TopicPartition, DescribeProducersResponseData.PartitionResponse> allResponses = Maps.newConcurrentMap(); |
| 2034 | + Map<TopicPartition, Errors> errors = Maps.newConcurrentMap(); |
| 2035 | + String namespacePrefix = currentNamespacePrefix(); |
| 2036 | + final int numPartitions = request.data().topics().stream() |
| 2037 | + .mapToInt(t->t.partitionIndexes().size()) |
| 2038 | + .sum(); |
| 2039 | + Runnable completeOne = () -> { |
| 2040 | + if (errors.size() + allResponses.size() != numPartitions) { |
| 2041 | + // not enough responses |
| 2042 | + return; |
| 2043 | + } |
| 2044 | + errors.forEach((topicPartition, tpErrors) -> { |
| 2045 | + DescribeProducersResponseData.PartitionResponse topicResponse = |
| 2046 | + new DescribeProducersResponseData.PartitionResponse() |
| 2047 | + .setPartitionIndex(topicPartition.partition()) |
| 2048 | + .setErrorCode(tpErrors.code()) |
| 2049 | + .setErrorMessage(tpErrors.message()); |
| 2050 | + allResponses.put(topicPartition, topicResponse); |
| 2051 | + }); |
| 2052 | + DescribeProducersResponseData response = new DescribeProducersResponseData(); |
| 2053 | + allResponses |
| 2054 | + .entrySet() |
| 2055 | + .stream() |
| 2056 | + .collect(Collectors.groupingBy( |
| 2057 | + entry -> entry.getKey().topic(), |
| 2058 | + Collectors.mapping( |
| 2059 | + entry -> entry.getValue(), |
| 2060 | + Collectors.toList() |
| 2061 | + ) |
| 2062 | + )) |
| 2063 | + .forEach((topic, partitionResponses) -> { |
| 2064 | + DescribeProducersResponseData.TopicResponse topicResponse = |
| 2065 | + new DescribeProducersResponseData.TopicResponse() |
| 2066 | + .setName(topic) |
| 2067 | + .setPartitions(partitionResponses); |
| 2068 | + response.topics().add(topicResponse); |
| 2069 | + }); |
| 2070 | + responseFuture.complete(new DescribeProducersResponse(response)); |
| 2071 | + }; |
| 2072 | + |
| 2073 | + request.data().topics().forEach ((topicRequest) -> { |
| 2074 | + topicRequest.partitionIndexes().forEach(partition -> { |
| 2075 | + TopicPartition tp = new TopicPartition(topicRequest.name(), partition); |
| 2076 | + String fullPartitionName; |
| 2077 | + try { |
| 2078 | + fullPartitionName = KopTopic.toString(tp, namespacePrefix); |
| 2079 | + } catch (KoPTopicException e) { |
| 2080 | + log.warn("Invalid topic name: {}", tp.topic(), e); |
| 2081 | + errors.put(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION); |
| 2082 | + completeOne.run(); |
| 2083 | + return; |
| 2084 | + } |
| 2085 | + authorize(AclOperation.WRITE, Resource.of(ResourceType.TOPIC, fullPartitionName)) |
| 2086 | + .whenComplete((isAuthorized, ex) -> { |
| 2087 | + if (ex != null) { |
| 2088 | + log.error("AddPartitionsToTxn topic authorize failed, topic - {}. {}", |
| 2089 | + fullPartitionName, ex.getMessage()); |
| 2090 | + errors.put(tp, Errors.TOPIC_AUTHORIZATION_FAILED); |
| 2091 | + completeOne.run(); |
| 2092 | + return; |
| 2093 | + } |
| 2094 | + if (!isAuthorized) { |
| 2095 | + errors.put(tp, Errors.TOPIC_AUTHORIZATION_FAILED); |
| 2096 | + completeOne.run(); |
| 2097 | + return; |
| 2098 | + } |
| 2099 | + CompletableFuture<DescribeProducersResponseData.PartitionResponse> topicResponse = |
| 2100 | + replicaManager.activeProducerState(tp, namespacePrefix); |
| 2101 | + topicResponse.whenComplete((response, throwable) -> { |
| 2102 | + if (throwable != null) { |
| 2103 | + log.error("DescribeProducersRequest failed, topic - {}. {}", |
| 2104 | + fullPartitionName, throwable.getMessage()); |
| 2105 | + errors.put(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION); |
| 2106 | + } else { |
| 2107 | + allResponses.put(tp, response); |
| 2108 | + } |
| 2109 | + completeOne.run(); |
| 2110 | + }); |
| 2111 | + |
| 2112 | + }); |
| 2113 | + }); |
| 2114 | + }); |
| 2115 | + |
| 2116 | + |
| 2117 | + } |
| 2118 | + |
2023 | 2119 | @Override
|
2024 | 2120 | protected void handleListGroupsRequest(KafkaHeaderAndRequest listGroups,
|
2025 | 2121 | CompletableFuture<AbstractResponse> resultFuture) {
|
|
0 commit comments