Browse Source

HDFS-17331:Fix Blocks are always -1 and DataNode version are always UNKNOWN in federationhealth.html (#6429). Contributed by lei w.

Signed-off-by: Shuyan Zhang <zhangshuyan@apache.org>
Lei313 1 year ago
parent
commit
cc4c4be1b7

+ 13 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java

@@ -143,7 +143,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
       final int xceiverCount, final String networkLocation,
       final int xceiverCount, final String networkLocation,
       final AdminStates adminState, final String upgradeDomain,
       final AdminStates adminState, final String upgradeDomain,
       final long lastBlockReportTime, final long lastBlockReportMonotonic,
       final long lastBlockReportTime, final long lastBlockReportMonotonic,
-                       final int blockCount) {
+      final int blockCount, final String softwareVersion) {
     super(ipAddr, hostName, datanodeUuid, xferPort, infoPort, infoSecurePort,
     super(ipAddr, hostName, datanodeUuid, xferPort, infoPort, infoSecurePort,
         ipcPort);
         ipcPort);
     this.capacity = capacity;
     this.capacity = capacity;
@@ -162,6 +162,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.lastBlockReportTime = lastBlockReportTime;
     this.lastBlockReportTime = lastBlockReportTime;
     this.lastBlockReportMonotonic = lastBlockReportMonotonic;
     this.lastBlockReportMonotonic = lastBlockReportMonotonic;
     this.numBlocks = blockCount;
     this.numBlocks = blockCount;
+    this.softwareVersion =  softwareVersion;
   }
   }
 
 
   /** Network location name. */
   /** Network location name. */
@@ -699,6 +700,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     private long lastBlockReportTime = 0L;
     private long lastBlockReportTime = 0L;
     private long lastBlockReportMonotonic = 0L;
     private long lastBlockReportMonotonic = 0L;
     private int numBlocks = 0;
     private int numBlocks = 0;
+    private String softwareVersion;
 
 
     // Please use setNumBlocks explicitly to set numBlocks as this method doesn't have
     // Please use setNumBlocks explicitly to set numBlocks as this method doesn't have
     // sufficient info about numBlocks
     // sufficient info about numBlocks
@@ -718,6 +720,9 @@ public class DatanodeInfo extends DatanodeID implements Node {
       this.upgradeDomain = from.getUpgradeDomain();
       this.upgradeDomain = from.getUpgradeDomain();
       this.lastBlockReportTime = from.getLastBlockReportTime();
       this.lastBlockReportTime = from.getLastBlockReportTime();
       this.lastBlockReportMonotonic = from.getLastBlockReportMonotonic();
       this.lastBlockReportMonotonic = from.getLastBlockReportMonotonic();
+      if (from.getSoftwareVersion() != null) {
+        this.softwareVersion = from.getSoftwareVersion();
+      }
       setNodeID(from);
       setNodeID(from);
       return this;
       return this;
     }
     }
@@ -844,18 +849,24 @@ public class DatanodeInfo extends DatanodeID implements Node {
       this.lastBlockReportMonotonic = time;
       this.lastBlockReportMonotonic = time;
       return this;
       return this;
     }
     }
+
     public DatanodeInfoBuilder setNumBlocks(int blockCount) {
     public DatanodeInfoBuilder setNumBlocks(int blockCount) {
       this.numBlocks = blockCount;
       this.numBlocks = blockCount;
       return this;
       return this;
     }
     }
 
 
+    public DatanodeInfoBuilder setSoftwareVersion(String dnVersion) {
+      this.softwareVersion = dnVersion;
+      return this;
+    }
+
     public DatanodeInfo build() {
     public DatanodeInfo build() {
       return new DatanodeInfo(ipAddr, hostName, datanodeUuid, xferPort,
       return new DatanodeInfo(ipAddr, hostName, datanodeUuid, xferPort,
           infoPort, infoSecurePort, ipcPort, capacity, dfsUsed, nonDfsUsed,
           infoPort, infoSecurePort, ipcPort, capacity, dfsUsed, nonDfsUsed,
           remaining, blockPoolUsed, cacheCapacity, cacheUsed, lastUpdate,
           remaining, blockPoolUsed, cacheCapacity, cacheUsed, lastUpdate,
           lastUpdateMonotonic, xceiverCount, location, adminState,
           lastUpdateMonotonic, xceiverCount, location, adminState,
           upgradeDomain, lastBlockReportTime, lastBlockReportMonotonic,
           upgradeDomain, lastBlockReportTime, lastBlockReportMonotonic,
-          numBlocks);
+          numBlocks, softwareVersion);
     }
     }
   }
   }
 }
 }

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -385,6 +385,9 @@ public class PBHelperClient {
     if (info.getUpgradeDomain() != null) {
     if (info.getUpgradeDomain() != null) {
       builder.setUpgradeDomain(info.getUpgradeDomain());
       builder.setUpgradeDomain(info.getUpgradeDomain());
     }
     }
+    if (info.getSoftwareVersion() != null) {
+      builder.setSoftwareVersion(info.getSoftwareVersion());
+    }
     builder
     builder
         .setId(convert((DatanodeID) info))
         .setId(convert((DatanodeID) info))
         .setCapacity(info.getCapacity())
         .setCapacity(info.getCapacity())
@@ -786,7 +789,8 @@ public class PBHelperClient {
                 di.getLastBlockReportTime() : 0)
                 di.getLastBlockReportTime() : 0)
             .setLastBlockReportMonotonic(di.hasLastBlockReportMonotonic() ?
             .setLastBlockReportMonotonic(di.hasLastBlockReportMonotonic() ?
                 di.getLastBlockReportMonotonic() : 0)
                 di.getLastBlockReportMonotonic() : 0)
-            .setNumBlocks(di.getNumBlocks());
+            .setNumBlocks(di.getNumBlocks())
+            .setSoftwareVersion(di.getSoftwareVersion());
 
 
     if (di.hasNonDfsUsed()) {
     if (di.hasNonDfsUsed()) {
       dinfo.setNonDfsUsed(di.getNonDfsUsed());
       dinfo.setNonDfsUsed(di.getNonDfsUsed());

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto

@@ -133,6 +133,7 @@ message DatanodeInfoProto {
   optional uint64 lastBlockReportTime = 15 [default = 0];
   optional uint64 lastBlockReportTime = 15 [default = 0];
   optional uint64 lastBlockReportMonotonic = 16 [default = 0];
   optional uint64 lastBlockReportMonotonic = 16 [default = 0];
   optional uint32 numBlocks = 17 [default = 0];
   optional uint32 numBlocks = 17 [default = 0];
+  optional string softwareVersion = 18;
 }
 }
 
 
 /**
 /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/NamenodeBeanMetrics.java

@@ -481,7 +481,7 @@ public class NamenodeBeanMetrics
         innerinfo.put("adminState", node.getAdminState().toString());
         innerinfo.put("adminState", node.getAdminState().toString());
         innerinfo.put("nonDfsUsedSpace", node.getNonDfsUsed());
         innerinfo.put("nonDfsUsedSpace", node.getNonDfsUsed());
         innerinfo.put("capacity", node.getCapacity());
         innerinfo.put("capacity", node.getCapacity());
-        innerinfo.put("numBlocks", -1); // node.numBlocks()
+        innerinfo.put("numBlocks", node.getNumBlocks());
         innerinfo.put("version", (node.getSoftwareVersion() == null ?
         innerinfo.put("version", (node.getSoftwareVersion() == null ?
                         "UNKNOWN" : node.getSoftwareVersion()));
                         "UNKNOWN" : node.getSoftwareVersion()));
         innerinfo.put("used", node.getDfsUsed());
         innerinfo.put("used", node.getDfsUsed());

+ 7 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java

@@ -1101,10 +1101,15 @@ public class RouterClientProtocol implements ClientProtocol {
         DatanodeInfo dnInfo = dn.getDatanodeInfo();
         DatanodeInfo dnInfo = dn.getDatanodeInfo();
         String nodeId = dnInfo.getXferAddr();
         String nodeId = dnInfo.getXferAddr();
         DatanodeStorageReport oldDn = datanodesMap.get(nodeId);
         DatanodeStorageReport oldDn = datanodesMap.get(nodeId);
-        if (oldDn == null ||
-            dnInfo.getLastUpdate() > oldDn.getDatanodeInfo().getLastUpdate()) {
+        if (oldDn == null) {
+          datanodesMap.put(nodeId, dn);
+        } else if (dnInfo.getLastUpdate() > oldDn.getDatanodeInfo().getLastUpdate()) {
+          dnInfo.setNumBlocks(dnInfo.getNumBlocks() +
+              oldDn.getDatanodeInfo().getNumBlocks());
           datanodesMap.put(nodeId, dn);
           datanodesMap.put(nodeId, dn);
         } else {
         } else {
+          oldDn.getDatanodeInfo().setNumBlocks(
+              oldDn.getDatanodeInfo().getNumBlocks() + dnInfo.getNumBlocks());
           LOG.debug("{} is in multiple subclusters", nodeId);
           LOG.debug("{} is in multiple subclusters", nodeId);
         }
         }
       }
       }