|
@@ -43,6 +43,7 @@ import static org.apache.hadoop.hdfs.DFSUtilClient.percent2String;
|
|
|
public class DatanodeInfo extends DatanodeID implements Node {
|
|
|
private long capacity;
|
|
|
private long dfsUsed;
|
|
|
+ private long nonDfsUsed;
|
|
|
private long remaining;
|
|
|
private long blockPoolUsed;
|
|
|
private long cacheCapacity;
|
|
@@ -89,6 +90,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|
|
super(from);
|
|
|
this.capacity = from.getCapacity();
|
|
|
this.dfsUsed = from.getDfsUsed();
|
|
|
+ this.nonDfsUsed = from.getNonDfsUsed();
|
|
|
this.remaining = from.getRemaining();
|
|
|
this.blockPoolUsed = from.getBlockPoolUsed();
|
|
|
this.cacheCapacity = from.getCacheCapacity();
|
|
@@ -105,6 +107,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|
|
super(nodeID);
|
|
|
this.capacity = 0L;
|
|
|
this.dfsUsed = 0L;
|
|
|
+ this.nonDfsUsed = 0L;
|
|
|
this.remaining = 0L;
|
|
|
this.blockPoolUsed = 0L;
|
|
|
this.cacheCapacity = 0L;
|
|
@@ -158,10 +161,26 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|
|
final int xceiverCount, final String networkLocation,
|
|
|
final AdminStates adminState,
|
|
|
final String upgradeDomain) {
|
|
|
- super(ipAddr, hostName, datanodeUuid, xferPort, infoPort,
|
|
|
- infoSecurePort, ipcPort);
|
|
|
+ this(ipAddr, hostName, datanodeUuid, xferPort, infoPort, infoSecurePort,
|
|
|
+ ipcPort, capacity, dfsUsed, 0L, remaining, blockPoolUsed,
|
|
|
+ cacheCapacity, cacheUsed, lastUpdate, lastUpdateMonotonic,
|
|
|
+ xceiverCount, networkLocation, adminState, upgradeDomain);
|
|
|
+ }
|
|
|
+
|
|
|
+ /** Constructor. */
|
|
|
+ public DatanodeInfo(final String ipAddr, final String hostName,
|
|
|
+ final String datanodeUuid, final int xferPort, final int infoPort,
|
|
|
+ final int infoSecurePort, final int ipcPort, final long capacity,
|
|
|
+ final long dfsUsed, final long nonDfsUsed, final long remaining,
|
|
|
+ final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
|
|
|
+ final long lastUpdate, final long lastUpdateMonotonic,
|
|
|
+ final int xceiverCount, final String networkLocation,
|
|
|
+ final AdminStates adminState, final String upgradeDomain) {
|
|
|
+ super(ipAddr, hostName, datanodeUuid, xferPort, infoPort, infoSecurePort,
|
|
|
+ ipcPort);
|
|
|
this.capacity = capacity;
|
|
|
this.dfsUsed = dfsUsed;
|
|
|
+ this.nonDfsUsed = nonDfsUsed;
|
|
|
this.remaining = remaining;
|
|
|
this.blockPoolUsed = blockPoolUsed;
|
|
|
this.cacheCapacity = cacheCapacity;
|
|
@@ -174,7 +193,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|
|
this.upgradeDomain = upgradeDomain;
|
|
|
}
|
|
|
|
|
|
- /** Network location name */
|
|
|
+ /** Network location name. */
|
|
|
@Override
|
|
|
public String getName() {
|
|
|
return getXferAddr();
|
|
@@ -191,8 +210,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|
|
|
|
|
/** The used space by the data node. */
|
|
|
public long getNonDfsUsed() {
|
|
|
- long nonDFSUsed = capacity - dfsUsed - remaining;
|
|
|
- return nonDFSUsed < 0 ? 0 : nonDFSUsed;
|
|
|
+ return nonDfsUsed;
|
|
|
}
|
|
|
|
|
|
/** The used space by the data node as percentage of present capacity */
|
|
@@ -282,6 +300,11 @@ public class DatanodeInfo extends DatanodeID implements Node {
|
|
|
this.dfsUsed = dfsUsed;
|
|
|
}
|
|
|
|
|
|
+ /** Sets the nondfs-used space for the datanode. */
|
|
|
+ public void setNonDfsUsed(long nonDfsUsed) {
|
|
|
+ this.nonDfsUsed = nonDfsUsed;
|
|
|
+ }
|
|
|
+
|
|
|
/** Sets raw free space. */
|
|
|
public void setRemaining(long remaining) {
|
|
|
this.remaining = remaining;
|