xBis7 commented on code in PR #4054:
URL: https://github.com/apache/ozone/pull/4054#discussion_r1053313749
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerMXBean.java:
##########
@@ -36,4 +36,7 @@ public interface PipelineManagerMXBean {
*/
Map<String, Integer> getPipelineInfo() throws NotLeaderException;
+ Map<String, Map<String, String[]>> getPipelineLeaders()
+ throws NotLeaderException;
Review Comment:
`throws NotLeaderException` is redundant.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java:
##########
@@ -117,4 +120,48 @@ static List<Pipeline> checkPipelineContainSameDatanodes(
p.sameDatanodes(pipeline)))
.collect(Collectors.toList());
}
+
+ /**
+ * Returns a map containing pipeline information which includes
+ * pipeline-id & the corresponding roles of all datanodes along with
+ * their respective datanode-id
+ * part of the pipeline.
+ *
+ * @param pipelines input pipeline
+ * @return map containing pipeline details
+ */
+ public static Map<String, Map<String, String[]>> pipelineLeaderFormat(
+ List<Pipeline> pipelines) {
+ final Map<String, Map<String, String[]>> pipelineInfo = new HashMap<>();
+
+ pipelines.forEach(pipeline -> {
+ String leaderNode = "";
+ List<DatanodeDetails> dataNodes = pipeline.getNodes();
+ UUID pipelineId = pipeline.getId().getId();
+
+ try {
+ leaderNode = pipeline.getLeaderNode().getHostName();
+ } catch (IOException ioEx) {
+ LOG.warn("Cannot get leader node for pipeline {}",
+ pipelineId, ioEx);
+ }
+
+ int numOfNodes = dataNodes.size();
+ Map<String, String[]> nodeInfo = new HashMap<>();
Review Comment:
```suggestion
Map<String, String> nodeInfo = new ConcurrentHashMap<>();
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java:
##########
@@ -117,4 +120,48 @@ static List<Pipeline> checkPipelineContainSameDatanodes(
p.sameDatanodes(pipeline)))
.collect(Collectors.toList());
}
+
+ /**
+ * Returns a map containing pipeline information which includes
+ * pipeline-id & the corresponding roles of all datanodes along with
+ * their respective datanode-id
+ * part of the pipeline.
+ *
+ * @param pipelines input pipeline
+ * @return map containing pipeline details
+ */
+ public static Map<String, Map<String, String[]>> pipelineLeaderFormat(
Review Comment:
We could simplify it and add the `hostname` or `role` or `uuid` as keys to
improve user readability.
```suggestion
public static Map<String, Map<String, String>> pipelineLeaderFormat(
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java:
##########
@@ -117,4 +120,48 @@ static List<Pipeline> checkPipelineContainSameDatanodes(
p.sameDatanodes(pipeline)))
.collect(Collectors.toList());
}
+
+ /**
+ * Returns a map containing pipeline information which includes
+ * pipeline-id & the corresponding roles of all datanodes along with
+ * their respective datanode-id
+ * part of the pipeline.
+ *
+ * @param pipelines input pipeline
+ * @return map containing pipeline details
+ */
+ public static Map<String, Map<String, String[]>> pipelineLeaderFormat(
+ List<Pipeline> pipelines) {
+ final Map<String, Map<String, String[]>> pipelineInfo = new HashMap<>();
+
+ pipelines.forEach(pipeline -> {
+ String leaderNode = "";
+ List<DatanodeDetails> dataNodes = pipeline.getNodes();
+ UUID pipelineId = pipeline.getId().getId();
+
+ try {
+ leaderNode = pipeline.getLeaderNode().getHostName();
+ } catch (IOException ioEx) {
+ LOG.warn("Cannot get leader node for pipeline {}",
+ pipelineId, ioEx);
+ }
+
+ int numOfNodes = dataNodes.size();
+ Map<String, String[]> nodeInfo = new HashMap<>();
+ for (int cnt = 0; cnt < numOfNodes; cnt++) {
+ String[] info = new String[2];
+ DatanodeDetails node = dataNodes.get(cnt);
+ String role =
+ node.getHostName().equals(leaderNode) ? "Leader" : "Follower";
+ String dataNodeUUID = node.getUuidString();
+ String hostName = node.getHostName();
+ info[0] = role;
+ info[1] = dataNodeUUID;
Review Comment:
```suggestion
nodeInfo.put("UUID", dataNodeUUID);
nodeInfo.put("HostName", hostName);
```
##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/MockPipelineManager.java:
##########
@@ -296,6 +296,11 @@ public Map<String, Integer> getPipelineInfo() {
return null;
}
+ @Override
+ public Map<String, Map<String, String[]>> getPipelineLeaders() {
Review Comment:
```suggestion
public Map<String, Map<String, String>> getPipelineLeaders() {
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java:
##########
@@ -697,6 +699,13 @@ public Map<String, Integer> getPipelineInfo() throws
NotLeaderException {
return pipelineInfo;
}
+ @Override
+ public Map<String, Map<String, String[]>> getPipelineLeaders()
Review Comment:
```suggestion
public Map<String, Map<String, String>> getPipelineLeaders()
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java:
##########
@@ -117,4 +120,48 @@ static List<Pipeline> checkPipelineContainSameDatanodes(
p.sameDatanodes(pipeline)))
.collect(Collectors.toList());
}
+
+ /**
+ * Returns a map containing pipeline information which includes
+ * pipeline-id & the corresponding roles of all datanodes along with
+ * their respective datanode-id
+ * part of the pipeline.
+ *
+ * @param pipelines input pipeline
+ * @return map containing pipeline details
+ */
+ public static Map<String, Map<String, String[]>> pipelineLeaderFormat(
+ List<Pipeline> pipelines) {
+ final Map<String, Map<String, String[]>> pipelineInfo = new HashMap<>();
+
+ pipelines.forEach(pipeline -> {
+ String leaderNode = "";
+ List<DatanodeDetails> dataNodes = pipeline.getNodes();
+ UUID pipelineId = pipeline.getId().getId();
+
+ try {
+ leaderNode = pipeline.getLeaderNode().getHostName();
+ } catch (IOException ioEx) {
+ LOG.warn("Cannot get leader node for pipeline {}",
+ pipelineId, ioEx);
+ }
+
+ int numOfNodes = dataNodes.size();
+ Map<String, String[]> nodeInfo = new HashMap<>();
+ for (int cnt = 0; cnt < numOfNodes; cnt++) {
+ String[] info = new String[2];
+ DatanodeDetails node = dataNodes.get(cnt);
+ String role =
+ node.getHostName().equals(leaderNode) ? "Leader" : "Follower";
+ String dataNodeUUID = node.getUuidString();
+ String hostName = node.getHostName();
+ info[0] = role;
Review Comment:
```suggestion
nodeInfo.put("Role", role);
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerImpl.java:
##########
@@ -697,6 +699,13 @@ public Map<String, Integer> getPipelineInfo() throws
NotLeaderException {
return pipelineInfo;
}
+ @Override
+ public Map<String, Map<String, String[]>> getPipelineLeaders()
+ throws NotLeaderException {
Review Comment:
`throws NotLeaderException` is redundant.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java:
##########
@@ -117,4 +120,48 @@ static List<Pipeline> checkPipelineContainSameDatanodes(
p.sameDatanodes(pipeline)))
.collect(Collectors.toList());
}
+
+ /**
+ * Returns a map containing pipeline information which includes
+ * pipeline-id & the corresponding roles of all datanodes along with
+ * their respective datanode-id
+ * part of the pipeline.
+ *
+ * @param pipelines input pipeline
+ * @return map containing pipeline details
+ */
+ public static Map<String, Map<String, String[]>> pipelineLeaderFormat(
+ List<Pipeline> pipelines) {
+ final Map<String, Map<String, String[]>> pipelineInfo = new HashMap<>();
+
+ pipelines.forEach(pipeline -> {
+ String leaderNode = "";
+ List<DatanodeDetails> dataNodes = pipeline.getNodes();
+ UUID pipelineId = pipeline.getId().getId();
+
+ try {
+ leaderNode = pipeline.getLeaderNode().getHostName();
+ } catch (IOException ioEx) {
+ LOG.warn("Cannot get leader node for pipeline {}",
+ pipelineId, ioEx);
+ }
+
+ int numOfNodes = dataNodes.size();
+ Map<String, String[]> nodeInfo = new HashMap<>();
+ for (int cnt = 0; cnt < numOfNodes; cnt++) {
+ String[] info = new String[2];
+ DatanodeDetails node = dataNodes.get(cnt);
+ String role =
+ node.getHostName().equals(leaderNode) ? "Leader" : "Follower";
+ String dataNodeUUID = node.getUuidString();
+ String hostName = node.getHostName();
+ info[0] = role;
+ info[1] = dataNodeUUID;
+ nodeInfo.put(hostName, info);
Review Comment:
This also becomes redundant.
```suggestion
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java:
##########
@@ -117,4 +120,48 @@ static List<Pipeline> checkPipelineContainSameDatanodes(
p.sameDatanodes(pipeline)))
.collect(Collectors.toList());
}
+
+ /**
+ * Returns a map containing pipeline information which includes
+ * pipeline-id & the corresponding roles of all datanodes along with
+ * their respective datanode-id
+ * part of the pipeline.
+ *
+ * @param pipelines input pipeline
+ * @return map containing pipeline details
+ */
+ public static Map<String, Map<String, String[]>> pipelineLeaderFormat(
+ List<Pipeline> pipelines) {
+ final Map<String, Map<String, String[]>> pipelineInfo = new HashMap<>();
Review Comment:
We could also change initialization to `new ConcurrentHashMap<>()` to
improve thread-safety.
```suggestion
final Map<String, Map<String, String>> pipelineInfo = new
ConcurrentHashMap<>();
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerMXBean.java:
##########
@@ -36,4 +36,7 @@ public interface PipelineManagerMXBean {
*/
Map<String, Integer> getPipelineInfo() throws NotLeaderException;
+ Map<String, Map<String, String[]>> getPipelineLeaders()
Review Comment:
```suggestion
Map<String, Map<String, String>> getPipelineLeaders()
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java:
##########
@@ -117,4 +120,48 @@ static List<Pipeline> checkPipelineContainSameDatanodes(
p.sameDatanodes(pipeline)))
.collect(Collectors.toList());
}
+
+ /**
+ * Returns a map containing pipeline information which includes
+ * pipeline-id & the corresponding roles of all datanodes along with
+ * their respective datanode-id
+ * part of the pipeline.
+ *
+ * @param pipelines input pipeline
+ * @return map containing pipeline details
+ */
+ public static Map<String, Map<String, String[]>> pipelineLeaderFormat(
+ List<Pipeline> pipelines) {
+ final Map<String, Map<String, String[]>> pipelineInfo = new HashMap<>();
+
+ pipelines.forEach(pipeline -> {
+ String leaderNode = "";
+ List<DatanodeDetails> dataNodes = pipeline.getNodes();
+ UUID pipelineId = pipeline.getId().getId();
+
+ try {
+ leaderNode = pipeline.getLeaderNode().getHostName();
+ } catch (IOException ioEx) {
+ LOG.warn("Cannot get leader node for pipeline {}",
+ pipelineId, ioEx);
+ }
+
+ int numOfNodes = dataNodes.size();
+ Map<String, String[]> nodeInfo = new HashMap<>();
+ for (int cnt = 0; cnt < numOfNodes; cnt++) {
+ String[] info = new String[2];
Review Comment:
This becomes redundant.
```suggestion
```
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]