diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto index 6cd75c7bf825..0085aa62466c 100644 --- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto +++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto @@ -202,6 +202,7 @@ message Node { repeated NodeOperationalState nodeOperationalStates = 3; optional int32 totalVolumeCount = 4; optional int32 healthyVolumeCount = 5; + repeated string failedVolumes = 6; } message NodePool { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java index 323b7dbe1e0c..48576ee71b20 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java @@ -166,6 +166,14 @@ public void getMetrics(MetricsCollector collector, boolean all) { Integer.parseInt(nonWritableNodes)); } + String volumeFailures = nodeStatistics.get("VolumeFailures"); + if (volumeFailures != null) { + metrics.addGauge( + Interns.info("VolumeFailures", + "Number of datanodes with at least one failed volume"), + Integer.parseInt(volumeFailures)); + } + for (Map.Entry e : nodeInfo.entrySet()) { metrics.addGauge( Interns.info(e.getKey(), diskMetricDescription(e.getKey())), diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index 3bd5214ea0ec..2c487b49f983 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -61,6 +61,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionInfo; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionSummary; import org.apache.hadoop.hdds.protocol.proto.ReconfigureProtocolProtos.ReconfigureProtocolService; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto; @@ -655,6 +656,7 @@ public List queryNode( if (datanodeInfo != null) { nodeBuilder.setTotalVolumeCount(datanodeInfo.getStorageReports().size()); nodeBuilder.setHealthyVolumeCount(datanodeInfo.getHealthyVolumeCount()); + addFailedVolumes(nodeBuilder, datanodeInfo); } result.add(nodeBuilder.build()); } @@ -687,6 +689,7 @@ public HddsProtos.Node queryNode(UUID uuid) if (datanodeInfo != null) { nodeBuilder.setTotalVolumeCount(datanodeInfo.getStorageReports().size()); nodeBuilder.setHealthyVolumeCount(datanodeInfo.getHealthyVolumeCount()); + addFailedVolumes(nodeBuilder, datanodeInfo); } result = nodeBuilder.build(); } @@ -702,6 +705,15 @@ public HddsProtos.Node queryNode(UUID uuid) return result; } + private static void addFailedVolumes(HddsProtos.Node.Builder nodeBuilder, + DatanodeInfo datanodeInfo) { + for (StorageReportProto report : datanodeInfo.getStorageReports()) { + if (report.hasFailed() && report.getFailed()) { + nodeBuilder.addFailedVolumes(report.getStorageLocation()); + } + } + } + @Override public List decommissionNodes(List nodes, boolean force) throws IOException { diff --git a/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/BasicDatanodeInfo.java b/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/BasicDatanodeInfo.java index ae0be5faab54..a439494cdd4e 100644 --- a/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/BasicDatanodeInfo.java +++ b/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/BasicDatanodeInfo.java @@ -20,6 +20,8 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -41,6 +43,8 @@ public final class BasicDatanodeInfo { private Integer totalVolumeCount = null; @JsonInclude(JsonInclude.Include.NON_NULL) private Integer healthyVolumeCount = null; + @JsonInclude(JsonInclude.Include.NON_EMPTY) + private List failedVolumes = null; private BasicDatanodeInfo(Builder builder) { this.dn = builder.dn; @@ -51,26 +55,30 @@ private BasicDatanodeInfo(Builder builder) { this.percentUsed = builder.percentUsed; this.totalVolumeCount = builder.totalVolumeCount; this.healthyVolumeCount = builder.healthyVolumeCount; + this.failedVolumes = builder.failedVolumes; } /** * Builder class for creating instances of BasicDatanodeInfo. */ public static class Builder { - private DatanodeDetails dn; - private HddsProtos.NodeOperationalState opState; - private HddsProtos.NodeState healthState; + private final DatanodeDetails dn; + private final HddsProtos.NodeOperationalState opState; + private final HddsProtos.NodeState healthState; private Long used; private Long capacity; private Double percentUsed; - private Integer totalVolumeCount; - private Integer healthyVolumeCount; - - public Builder(DatanodeDetails dn, HddsProtos.NodeOperationalState opState, - HddsProtos.NodeState healthState) { - this.dn = dn; - this.opState = opState; - this.healthState = healthState; + private final Integer totalVolumeCount; + private final Integer healthyVolumeCount; + private final List failedVolumes; + + public Builder(HddsProtos.Node node) { + dn = DatanodeDetails.getFromProtoBuf(node.getNodeID()); + healthState = node.getNodeStates(0); + opState = node.getNodeOperationalStates(0); + totalVolumeCount = node.hasTotalVolumeCount() ? node.getTotalVolumeCount() : null; + healthyVolumeCount = node.hasHealthyVolumeCount() ? node.getHealthyVolumeCount() : null; + failedVolumes = getFailedVolumes(node); } public Builder withUsageInfo(long usedBytes, long capacityBytes, double percentUsedBytes) { @@ -80,12 +88,6 @@ public Builder withUsageInfo(long usedBytes, long capacityBytes, double percentU return this; } - public Builder withVolumeCounts(Integer total, Integer healthy) { - this.totalVolumeCount = total; - this.healthyVolumeCount = healthy; - return this; - } - public BasicDatanodeInfo build() { return new BasicDatanodeInfo(this); } @@ -206,8 +208,26 @@ public Integer getHealthyVolumeCount() { return healthyVolumeCount; } + @JsonProperty(index = 112) + public List getFailedVolumes() { + return failedVolumes; + } + @JsonIgnore public DatanodeDetails getDatanodeDetails() { return dn; } + + private static List getFailedVolumes(HddsProtos.Node node) { + int count = node.getFailedVolumesCount(); + if (count == 0) { + return Collections.emptyList(); + } + List result = new ArrayList<>(count); + for (int i = 0; i < count; i++) { + result.add(node.getFailedVolumes(i)); + } + return result; + } + } diff --git a/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java b/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java index 0ed71c99e708..2ca9e906d96e 100644 --- a/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java +++ b/hadoop-ozone/cli-admin/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java @@ -65,6 +65,11 @@ public class ListInfoSubcommand extends ScmSubcommand { defaultValue = "false") private boolean json; + @CommandLine.Option(names = {"--nodes-with-failed-volumes"}, + description = "Only show datanodes that have at least one failed volume.", + defaultValue = "false") + private boolean nodeWithFailedVolumes; + @CommandLine.ArgGroup(exclusive = true, multiplicity = "0..1") private ExclusiveNodeOptions exclusiveNodeOptions; @@ -85,14 +90,16 @@ static class ExclusiveNodeOptions extends NodeSelectionMixin { @Override public void execute(ScmClient scmClient) throws IOException { + if (nodeWithFailedVolumes && exclusiveNodeOptions != null + && !Strings.isNullOrEmpty(exclusiveNodeOptions.getNodeId())) { + throw new IOException( + "--nodes-with-failed-volumes cannot be used with --id/--node-id. " + + "Use them separately."); + } pipelines = scmClient.listPipelines(); if (exclusiveNodeOptions != null && !Strings.isNullOrEmpty(exclusiveNodeOptions.getNodeId())) { HddsProtos.Node node = scmClient.queryNode(UUID.fromString(exclusiveNodeOptions.getNodeId())); - Integer totalVolumeCount = node.hasTotalVolumeCount() ? node.getTotalVolumeCount() : null; - Integer healthyVolumeCount = node.hasHealthyVolumeCount() ? node.getHealthyVolumeCount() : null; - BasicDatanodeInfo singleNodeInfo = new BasicDatanodeInfo.Builder( - DatanodeDetails.getFromProtoBuf(node.getNodeID()), node.getNodeOperationalStates(0), - node.getNodeStates(0)).withVolumeCounts(totalVolumeCount, healthyVolumeCount).build(); + BasicDatanodeInfo singleNodeInfo = new BasicDatanodeInfo.Builder(node).build(); if (json) { List dtoList = Collections.singletonList(singleNodeInfo); System.out.println(JsonUtils.toJsonStringWithDefaultPrettyPrinter(dtoList)); @@ -118,6 +125,10 @@ public void execute(ScmClient scmClient) throws IOException { allNodes = allNodes.filter(p -> p.getHealthState().toString() .compareToIgnoreCase(nodeState) == 0); } + if (nodeWithFailedVolumes) { + allNodes = allNodes.filter(p -> + p.getFailedVolumes() != null && !p.getFailedVolumes().isEmpty()); + } if (!listLimitOptions.isAll()) { allNodes = allNodes.limit(listLimitOptions.getLimit()); @@ -154,13 +165,9 @@ private List getAllNodes(ScmClient scmClient) long capacity = p.getCapacity(); long used = capacity - p.getRemaining(); double percentUsed = (capacity > 0) ? (used * 100.0) / capacity : 0.0; - Integer totalVolumeCount = node.hasTotalVolumeCount() ? node.getTotalVolumeCount() : null; - Integer healthyVolumeCount = node.hasHealthyVolumeCount() ? node.getHealthyVolumeCount() : null; - return new BasicDatanodeInfo.Builder( - DatanodeDetails.getFromProtoBuf(node.getNodeID()), - node.getNodeOperationalStates(0), node.getNodeStates(0)) + return new BasicDatanodeInfo.Builder(node) .withUsageInfo(used, capacity, percentUsed) - .withVolumeCounts(totalVolumeCount, healthyVolumeCount).build(); + .build(); } catch (Exception e) { String reason = "Could not process info for an unknown datanode"; if (p != null && p.getNode() != null && !Strings.isNullOrEmpty(p.getNode().getUuid())) { @@ -177,12 +184,7 @@ private List getAllNodes(ScmClient scmClient) List nodes = scmClient.queryNode(null, null, HddsProtos.QueryScope.CLUSTER, ""); - return nodes.stream().map(p -> { - Integer totalVolumeCount = p.hasTotalVolumeCount() ? p.getTotalVolumeCount() : null; - Integer healthyVolumeCount = p.hasHealthyVolumeCount() ? p.getHealthyVolumeCount() : null; - return new BasicDatanodeInfo.Builder( - DatanodeDetails.getFromProtoBuf(p.getNodeID()), p.getNodeOperationalStates(0), p.getNodeStates(0)) - .withVolumeCounts(totalVolumeCount, healthyVolumeCount).build(); }) + return nodes.stream().map(p -> new BasicDatanodeInfo.Builder(p).build()) .sorted(Comparator.comparing(BasicDatanodeInfo::getHealthState)) .collect(Collectors.toList()); } @@ -206,10 +208,12 @@ private void printDatanodeInfo(BasicDatanodeInfo dn) { .append('/').append(p.getPipelineState().toString()).append('/') .append(datanode.getID().equals(p.getLeaderId()) ? "Leader" : "Follower") - .append(System.getProperty("line.separator"))); + .append('\n')); } } else { - pipelineListInfo.append("No pipelines in cluster."); + pipelineListInfo + .append("No pipelines in cluster.") + .append('\n'); } System.out.println("Datanode: " + datanode.getUuid().toString() + " (" + datanode.getNetworkLocation() + "/" + datanode.getIpAddress() @@ -221,6 +225,12 @@ private void printDatanodeInfo(BasicDatanodeInfo dn) { System.out.println("Total volume count: " + dn.getTotalVolumeCount() + "\n" + "Healthy volume count: " + dn.getHealthyVolumeCount()); } + if (dn.getFailedVolumes() != null && !dn.getFailedVolumes().isEmpty()) { + System.out.println("Failed volumes:"); + for (String vol : dn.getFailedVolumes()) { + System.out.println(" " + vol); + } + } System.out.println("Related pipelines:\n" + pipelineListInfo); if (dn.getUsed() != null && dn.getCapacity() != null && dn.getUsed() >= 0 && dn.getCapacity() > 0) { diff --git a/hadoop-ozone/cli-admin/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestListInfoSubcommand.java b/hadoop-ozone/cli-admin/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestListInfoSubcommand.java index 13ae6a35f10d..dd0646034b71 100644 --- a/hadoop-ozone/cli-admin/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestListInfoSubcommand.java +++ b/hadoop-ozone/cli-admin/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestListInfoSubcommand.java @@ -17,6 +17,7 @@ package org.apache.hadoop.hdds.scm.cli.datanode; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -357,6 +358,62 @@ public void testVolumeCounters() throws Exception { assertTrue(output.contains("Healthy volume count:"), "Should display healthy volume count"); } + @Test + public void testFailedVolumesFilter() throws Exception { + ScmClient scmClient = mock(ScmClient.class); + List baseNodes = getNodeDetails(); + + List nodes = new ArrayList<>(); + // node 0: 1 failed volume + nodes.add(HddsProtos.Node.newBuilder(baseNodes.get(0)) + .setTotalVolumeCount(4).setHealthyVolumeCount(3) + .addFailedVolumes("/data/disk2").build()); + // node 1: healthy, no failed volumes + nodes.add(HddsProtos.Node.newBuilder(baseNodes.get(1)) + .setTotalVolumeCount(4).setHealthyVolumeCount(4).build()); + // node 2: 2 failed volumes + nodes.add(HddsProtos.Node.newBuilder(baseNodes.get(2)) + .setTotalVolumeCount(6).setHealthyVolumeCount(4) + .addFailedVolumes("/data/disk1") + .addFailedVolumes("/data/disk5").build()); + // node 3: healthy, no failed volumes + nodes.add(HddsProtos.Node.newBuilder(baseNodes.get(3)) + .setTotalVolumeCount(4).setHealthyVolumeCount(4).build()); + + when(scmClient.queryNode(any(), any(), any(), any())).thenReturn(nodes); + when(scmClient.listPipelines()).thenReturn(new ArrayList<>()); + + CommandLine c = new CommandLine(cmd); + c.parseArgs("--nodes-with-failed-volumes"); + cmd.execute(scmClient); + String output = outContent.toString(DEFAULT_ENCODING); + + // Only 2 datanodes (those with failed volumes) should appear + Matcher m = Pattern.compile("^Datanode:", Pattern.MULTILINE) + .matcher(output); + int count = 0; + while (m.find()) { + count++; + } + assertEquals(2, count, "Only datanodes with failed volumes should be listed"); + assertThat(output).contains("Failed volume"); + assertThat(output).contains("/data/disk2"); + assertThat(output).contains("/data/disk1"); + assertThat(output).contains("/data/disk5"); + } + + @Test + public void testFailedVolumesFilterRejectsNodeId() throws Exception { + ScmClient scmClient = mock(ScmClient.class); + List nodes = getNodeDetails(); + when(scmClient.listPipelines()).thenReturn(new ArrayList<>()); + + CommandLine c = new CommandLine(cmd); + c.parseArgs("--nodes-with-failed-volumes", + "--id", nodes.get(0).getNodeID().getUuid()); + assertThrows(IOException.class, () -> cmd.execute(scmClient)); + } + private void validateOrdering(JsonNode root, String orderDirection) { for (int i = 0; i < root.size() - 1; i++) { long usedCurrent = root.get(i).get("used").asLong();