|
2 | 2 |
|
3 | 3 | import cn.typesafe.km.entity.Cluster; |
4 | 4 | import cn.typesafe.km.service.dto.*; |
5 | | -import org.apache.kafka.clients.admin.AdminClient; |
6 | | -import org.apache.kafka.clients.admin.ConsumerGroupDescription; |
7 | | -import org.apache.kafka.clients.admin.ConsumerGroupListing; |
| 5 | +import org.apache.kafka.clients.admin.*; |
8 | 6 | import org.apache.kafka.clients.consumer.KafkaConsumer; |
9 | 7 | import org.apache.kafka.clients.consumer.OffsetAndMetadata; |
10 | 8 | import org.apache.kafka.common.TopicPartition; |
@@ -151,15 +149,28 @@ public List<ConsumerGroup> consumerGroup(String clusterId, String filterGroupId) |
151 | 149 | .map(groupId -> { |
152 | 150 | ConsumerGroup consumerGroup = new ConsumerGroup(); |
153 | 151 | consumerGroup.setGroupId(groupId); |
154 | | - |
155 | 152 | ConsumerGroupDescription consumerGroupDescription = groups.get(groupId); |
156 | | - Set<String> topics = consumerGroupDescription.members() |
157 | | - .stream() |
158 | | - .map(s -> s.assignment().topicPartitions()) |
159 | | - .flatMap(Collection::stream) |
160 | | - .map(TopicPartition::topic) |
161 | | - .collect(Collectors.toSet()); |
162 | | - consumerGroup.setTopics(topics); |
| 153 | + if (!consumerGroupDescription.isSimpleConsumerGroup()) { |
| 154 | + Set<String> topics = consumerGroupDescription.members() |
| 155 | + .stream() |
| 156 | + .map(s -> s.assignment().topicPartitions()) |
| 157 | + .flatMap(Collection::stream) |
| 158 | + .map(TopicPartition::topic) |
| 159 | + .collect(Collectors.toSet()); |
| 160 | + consumerGroup.setTopics(topics); |
| 161 | + return consumerGroup; |
| 162 | + } |
| 163 | + Map<TopicPartition, OffsetAndMetadata> topicPartitionOffsetAndMetadataMap=null; |
| 164 | + try { |
| 165 | + topicPartitionOffsetAndMetadataMap = adminClient.listConsumerGroupOffsets(groupId) |
| 166 | + .partitionsToOffsetAndMetadata() |
| 167 | + .get(); |
| 168 | + } catch (InterruptedException e) { |
| 169 | + throw new RuntimeException(e); |
| 170 | + } catch (ExecutionException e) { |
| 171 | + throw new RuntimeException(e); |
| 172 | + } |
| 173 | + consumerGroup.setTopics(topicPartitionOffsetAndMetadataMap.keySet().stream().map(TopicPartition::topic).collect(Collectors.toSet())); |
163 | 174 | return consumerGroup; |
164 | 175 | }) |
165 | 176 | .collect(Collectors.toList()); |
@@ -190,53 +201,84 @@ public void delete(String clusterId, String groupId) throws ExecutionException, |
190 | 201 | public List<ConsumerGroupDescribe> describe(String clusterId, String groupId) throws ExecutionException, InterruptedException { |
191 | 202 | AdminClient adminClient = clusterService.getAdminClient(clusterId); |
192 | 203 | ConsumerGroupDescription consumerGroupDescription = adminClient.describeConsumerGroups(Collections.singletonList(groupId)).all().get().get(groupId); |
193 | | - |
194 | 204 | Map<TopicPartition, OffsetAndMetadata> topicPartitionOffsetAndMetadataMap = adminClient.listConsumerGroupOffsets(groupId) |
195 | 205 | .partitionsToOffsetAndMetadata() |
196 | 206 | .get(); |
197 | 207 | Set<TopicPartition> topicPartitions = topicPartitionOffsetAndMetadataMap.keySet(); |
198 | 208 | try (KafkaConsumer<String, String> kafkaConsumer = clusterService.createConsumer(clusterId)) { |
199 | 209 | Map<TopicPartition, Long> endOffsets = kafkaConsumer.endOffsets(topicPartitions); |
200 | 210 | Map<TopicPartition, Long> beginningOffsets = kafkaConsumer.beginningOffsets(topicPartitions); |
201 | | - |
202 | | - return consumerGroupDescription.members() |
203 | | - .stream() |
204 | | - .flatMap(consumer -> { |
205 | | - return consumer.assignment().topicPartitions() |
206 | | - .stream() |
207 | | - .map(topicPartition -> { |
208 | | - String topic = topicPartition.topic(); |
209 | | - int partition = topicPartition.partition(); |
210 | | - String consumerId = consumer.consumerId(); |
211 | | - String host = consumer.host(); |
212 | | - String clientId = consumer.clientId(); |
213 | | - OffsetAndMetadata offsetAndMetadata = topicPartitionOffsetAndMetadataMap.get(topicPartition); |
214 | | - Long beginningOffset = beginningOffsets.get(topicPartition); |
215 | | - Long endOffset = endOffsets.get(topicPartition); |
216 | | - Long offset = null; |
217 | | - if (offsetAndMetadata != null) { |
218 | | - offset = offsetAndMetadata.offset(); |
219 | | - } |
220 | | - ConsumerGroupDescribe consumerGroupDescribe = new ConsumerGroupDescribe(); |
221 | | - consumerGroupDescribe.setGroupId(groupId); |
222 | | - consumerGroupDescribe.setTopic(topic); |
223 | | - consumerGroupDescribe.setPartition(partition); |
224 | | - consumerGroupDescribe.setCurrentOffset(offset); |
225 | | - consumerGroupDescribe.setLogBeginningOffset(beginningOffset); |
226 | | - consumerGroupDescribe.setLogEndOffset(endOffset); |
227 | | - if (endOffset != null && offset != null) { |
228 | | - consumerGroupDescribe.setLag(endOffset - offset); |
229 | | - } else { |
230 | | - consumerGroupDescribe.setLag(null); |
231 | | - } |
232 | | - consumerGroupDescribe.setConsumerId(consumerId); |
233 | | - consumerGroupDescribe.setHost(host); |
234 | | - consumerGroupDescribe.setClientId(clientId); |
235 | | - |
236 | | - return consumerGroupDescribe; |
237 | | - }); |
238 | | - }) |
239 | | - .collect(Collectors.toList()); |
| 211 | + if (!consumerGroupDescription.isSimpleConsumerGroup()) { |
| 212 | + return consumerGroupDescription.members() |
| 213 | + .stream() |
| 214 | + .flatMap(consumer -> { |
| 215 | + return consumer.assignment().topicPartitions() |
| 216 | + .stream() |
| 217 | + .map(topicPartition -> { |
| 218 | + String topic = topicPartition.topic(); |
| 219 | + int partition = topicPartition.partition(); |
| 220 | + String consumerId = consumer.consumerId(); |
| 221 | + String host = consumer.host(); |
| 222 | + String clientId = consumer.clientId(); |
| 223 | + OffsetAndMetadata offsetAndMetadata = topicPartitionOffsetAndMetadataMap.get(topicPartition); |
| 224 | + Long beginningOffset = beginningOffsets.get(topicPartition); |
| 225 | + Long endOffset = endOffsets.get(topicPartition); |
| 226 | + Long offset = null; |
| 227 | + if (offsetAndMetadata != null) { |
| 228 | + offset = offsetAndMetadata.offset(); |
| 229 | + } |
| 230 | + ConsumerGroupDescribe consumerGroupDescribe = new ConsumerGroupDescribe(); |
| 231 | + consumerGroupDescribe.setGroupId(groupId); |
| 232 | + consumerGroupDescribe.setTopic(topic); |
| 233 | + consumerGroupDescribe.setPartition(partition); |
| 234 | + consumerGroupDescribe.setCurrentOffset(offset); |
| 235 | + consumerGroupDescribe.setLogBeginningOffset(beginningOffset); |
| 236 | + consumerGroupDescribe.setLogEndOffset(endOffset); |
| 237 | + if (endOffset != null && offset != null) { |
| 238 | + consumerGroupDescribe.setLag(endOffset - offset); |
| 239 | + } else { |
| 240 | + consumerGroupDescribe.setLag(null); |
| 241 | + } |
| 242 | + consumerGroupDescribe.setConsumerId(consumerId); |
| 243 | + consumerGroupDescribe.setHost(host); |
| 244 | + consumerGroupDescribe.setClientId(clientId); |
| 245 | + return consumerGroupDescribe; |
| 246 | + }); |
| 247 | + }) |
| 248 | + .collect(Collectors.toList()); |
| 249 | + } |
| 250 | + List<ConsumerGroupDescribe> results=new ArrayList<>(); |
| 251 | + for (TopicPartition topicPartition : topicPartitionOffsetAndMetadataMap.keySet()) { |
| 252 | + String topic = topicPartition.topic(); |
| 253 | + int partition = topicPartition.partition(); |
| 254 | + //String consumerId = consumer.consumerId(); |
| 255 | + //String host = consumer.host(); |
| 256 | + //String clientId = consumer.clientId(); |
| 257 | + OffsetAndMetadata offsetAndMetadata = topicPartitionOffsetAndMetadataMap.get(topicPartition); |
| 258 | + Long beginningOffset = beginningOffsets.get(topicPartition); |
| 259 | + Long endOffset = endOffsets.get(topicPartition); |
| 260 | + Long offset = null; |
| 261 | + if (offsetAndMetadata != null) { |
| 262 | + offset = offsetAndMetadata.offset(); |
| 263 | + } |
| 264 | + ConsumerGroupDescribe consumerGroupDescribe = new ConsumerGroupDescribe(); |
| 265 | + consumerGroupDescribe.setGroupId(groupId); |
| 266 | + consumerGroupDescribe.setTopic(topic); |
| 267 | + consumerGroupDescribe.setPartition(partition); |
| 268 | + consumerGroupDescribe.setCurrentOffset(offset); |
| 269 | + consumerGroupDescribe.setLogBeginningOffset(beginningOffset); |
| 270 | + consumerGroupDescribe.setLogEndOffset(endOffset); |
| 271 | + if (endOffset != null && offset != null) { |
| 272 | + consumerGroupDescribe.setLag(endOffset - offset); |
| 273 | + } else { |
| 274 | + consumerGroupDescribe.setLag(null); |
| 275 | + } |
| 276 | + //consumerGroupDescribe.setConsumerId(consumerId); |
| 277 | + //consumerGroupDescribe.setHost(host); |
| 278 | + //consumerGroupDescribe.setClientId(clientId); |
| 279 | + results.add( consumerGroupDescribe); |
| 280 | + } |
| 281 | + return results; |
240 | 282 | } |
241 | 283 | } |
242 | 284 | } |
0 commit comments