chia7712 commented on code in PR #21222:
URL: https://github.com/apache/kafka/pull/21222#discussion_r2656495286
##########
tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java:
##########
@@ -565,15 +565,19 @@ public static Entry<Map<TopicPartition, List<Integer>>,
Map<TopicPartition, List
List<Integer> brokersToReassign = t0.getKey();
List<String> topicsToReassign = t0.getValue();
- Map<TopicPartition, List<Integer>> currentAssignments =
getReplicaAssignmentForTopics(adminClient, topicsToReassign);
- Map<TopicPartitionReplica, String> currentReplicaLogDirs =
getReplicaToLogDir(adminClient, currentAssignments);
+ Map<TopicPartition, List<Node>> currentAssignments =
getReplicaAssignmentForTopics(adminClient, topicsToReassign);
+ Map<TopicPartition, List<Integer>> currentParts =
toReplicaIds(currentAssignments);
+ Map<TopicPartition, List<Integer>> currentActiveParts = new
HashMap<>();
Review Comment:
The style feels a bit awkward, since we have to create an extra collection
before we use it. Maybe we can add a helper method instead. For example:
```java
static List<TopicPartitionReplica> available(Map<TopicPartition,
List<Node>> current) {
return current.entrySet().stream().flatMap(entry -> entry.getValue()
.stream().filter(n -> !n.isEmpty()).map(n -> new
TopicPartitionReplica(entry.getKey().topic(), entry.getKey().partition(),
n.id())))
.toList();
}
static Map<TopicPartitionReplica, String> getReplicaToLogDir(
Admin adminClient, Map<TopicPartition, List<Node>> current)
throws ExecutionException, InterruptedException {
return
adminClient.describeReplicaLogDirs(available(current)).all().get().entrySet()
.stream()
.filter(e -> e.getValue().getCurrentReplicaLogDir() != null)
.collect(Collectors.toMap(Entry::getKey, e ->
e.getValue().getCurrentReplicaLogDir()));
}
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]