HDFS-9371. Code cleanup for DatanodeManager. Contributed by Jing Zhao.
This commit is contained in:
parent
0c3a53e5a9
commit
8602692338
|
@ -899,6 +899,8 @@ Release 2.9.0 - UNRELEASED
|
||||||
HDFS-9281. Change TestDeleteBlockPool to not explicitly use File to check
|
HDFS-9281. Change TestDeleteBlockPool to not explicitly use File to check
|
||||||
block pool existence. (lei)
|
block pool existence. (lei)
|
||||||
|
|
||||||
|
HDFS-9371. Code cleanup for DatanodeManager. (jing9)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
BUG FIXES
|
BUG FIXES
|
||||||
|
|
|
@ -132,6 +132,9 @@ public class BlockManager implements BlockStatsMXBean {
|
||||||
private final HeartbeatManager heartbeatManager;
|
private final HeartbeatManager heartbeatManager;
|
||||||
private final BlockTokenSecretManager blockTokenSecretManager;
|
private final BlockTokenSecretManager blockTokenSecretManager;
|
||||||
|
|
||||||
|
// Block pool ID used by this namenode
|
||||||
|
private String blockPoolId;
|
||||||
|
|
||||||
private final PendingDataNodeMessages pendingDNMessages =
|
private final PendingDataNodeMessages pendingDNMessages =
|
||||||
new PendingDataNodeMessages();
|
new PendingDataNodeMessages();
|
||||||
|
|
||||||
|
@ -462,11 +465,16 @@ public class BlockManager implements BlockStatsMXBean {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setBlockPoolId(String blockPoolId) {
|
public void setBlockPoolId(String blockPoolId) {
|
||||||
|
this.blockPoolId = blockPoolId;
|
||||||
if (isBlockTokenEnabled()) {
|
if (isBlockTokenEnabled()) {
|
||||||
blockTokenSecretManager.setBlockPoolId(blockPoolId);
|
blockTokenSecretManager.setBlockPoolId(blockPoolId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public String getBlockPoolId() {
|
||||||
|
return blockPoolId;
|
||||||
|
}
|
||||||
|
|
||||||
public BlockStoragePolicySuite getStoragePolicySuite() {
|
public BlockStoragePolicySuite getStoragePolicySuite() {
|
||||||
return storagePolicySuite;
|
return storagePolicySuite;
|
||||||
}
|
}
|
||||||
|
@ -1228,18 +1236,6 @@ public class BlockManager implements BlockStatsMXBean {
|
||||||
((BlockInfoStriped) storedBlock).getBlockOnStorage(storage) : storedBlock;
|
((BlockInfoStriped) storedBlock).getBlockOnStorage(storage) : storedBlock;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Remove all block invalidation tasks under this datanode UUID;
|
|
||||||
* used when a datanode registers with a new UUID and the old one
|
|
||||||
* is wiped.
|
|
||||||
*/
|
|
||||||
void removeFromInvalidates(final DatanodeInfo datanode) {
|
|
||||||
if (!isPopulatingReplQueues()) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
invalidateBlocks.remove(datanode);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mark the block belonging to datanode as corrupt
|
* Mark the block belonging to datanode as corrupt
|
||||||
* @param blk Block to be marked as corrupt
|
* @param blk Block to be marked as corrupt
|
||||||
|
|
|
@ -290,11 +290,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
||||||
this.isAlive = isAlive;
|
this.isAlive = isAlive;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean needKeyUpdate() {
|
public synchronized boolean needKeyUpdate() {
|
||||||
return needKeyUpdate;
|
return needKeyUpdate;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setNeedKeyUpdate(boolean needKeyUpdate) {
|
public synchronized void setNeedKeyUpdate(boolean needKeyUpdate) {
|
||||||
this.needKeyUpdate = needKeyUpdate;
|
this.needKeyUpdate = needKeyUpdate;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -868,14 +868,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
||||||
/**
|
/**
|
||||||
* @return balancer bandwidth in bytes per second for this datanode
|
* @return balancer bandwidth in bytes per second for this datanode
|
||||||
*/
|
*/
|
||||||
public long getBalancerBandwidth() {
|
public synchronized long getBalancerBandwidth() {
|
||||||
return this.bandwidth;
|
return this.bandwidth;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param bandwidth balancer bandwidth in bytes per second for this datanode
|
* @param bandwidth balancer bandwidth in bytes per second for this datanode
|
||||||
*/
|
*/
|
||||||
public void setBalancerBandwidth(long bandwidth) {
|
public synchronized void setBalancerBandwidth(long bandwidth) {
|
||||||
this.bandwidth = bandwidth;
|
this.bandwidth = bandwidth;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -17,6 +17,7 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hdfs.server.blockmanagement;
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY;
|
||||||
import static org.apache.hadoop.util.Time.monotonicNow;
|
import static org.apache.hadoop.util.Time.monotonicNow;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
@ -162,7 +163,7 @@ public class DatanodeManager {
|
||||||
* during rolling upgrades.
|
* during rolling upgrades.
|
||||||
* Software version -> Number of datanodes with this version
|
* Software version -> Number of datanodes with this version
|
||||||
*/
|
*/
|
||||||
private HashMap<String, Integer> datanodesSoftwareVersions =
|
private final HashMap<String, Integer> datanodesSoftwareVersions =
|
||||||
new HashMap<>(4, 0.75f);
|
new HashMap<>(4, 0.75f);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -352,15 +353,9 @@ public class DatanodeManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean isInactive(DatanodeInfo datanode) {
|
private boolean isInactive(DatanodeInfo datanode) {
|
||||||
if (datanode.isDecommissioned()) {
|
return datanode.isDecommissioned() ||
|
||||||
return true;
|
(avoidStaleDataNodesForRead && datanode.isStale(staleInterval));
|
||||||
}
|
|
||||||
|
|
||||||
if (avoidStaleDataNodesForRead) {
|
|
||||||
return datanode.isStale(staleInterval);
|
|
||||||
}
|
|
||||||
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Sort the located blocks by the distance to the target host. */
|
/** Sort the located blocks by the distance to the target host. */
|
||||||
|
@ -479,8 +474,9 @@ public class DatanodeManager {
|
||||||
if (datanodeUuid == null) {
|
if (datanodeUuid == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
synchronized (this) {
|
||||||
return datanodeMap.get(datanodeUuid);
|
return datanodeMap.get(datanodeUuid);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -490,8 +486,8 @@ public class DatanodeManager {
|
||||||
* @return DatanodeDescriptor or null if the node is not found.
|
* @return DatanodeDescriptor or null if the node is not found.
|
||||||
* @throws UnregisteredNodeException
|
* @throws UnregisteredNodeException
|
||||||
*/
|
*/
|
||||||
public DatanodeDescriptor getDatanode(DatanodeID nodeID
|
public DatanodeDescriptor getDatanode(DatanodeID nodeID)
|
||||||
) throws UnregisteredNodeException {
|
throws UnregisteredNodeException {
|
||||||
final DatanodeDescriptor node = getDatanode(nodeID.getDatanodeUuid());
|
final DatanodeDescriptor node = getDatanode(nodeID.getDatanodeUuid());
|
||||||
if (node == null)
|
if (node == null)
|
||||||
return null;
|
return null;
|
||||||
|
@ -535,13 +531,13 @@ public class DatanodeManager {
|
||||||
|
|
||||||
/** Prints information about all datanodes. */
|
/** Prints information about all datanodes. */
|
||||||
void datanodeDump(final PrintWriter out) {
|
void datanodeDump(final PrintWriter out) {
|
||||||
synchronized (datanodeMap) {
|
final Map<String,DatanodeDescriptor> sortedDatanodeMap;
|
||||||
Map<String,DatanodeDescriptor> sortedDatanodeMap =
|
synchronized (this) {
|
||||||
new TreeMap<>(datanodeMap);
|
sortedDatanodeMap = new TreeMap<>(datanodeMap);
|
||||||
out.println("Metasave: Number of datanodes: " + datanodeMap.size());
|
}
|
||||||
for (DatanodeDescriptor node : sortedDatanodeMap.values()) {
|
out.println("Metasave: Number of datanodes: " + sortedDatanodeMap.size());
|
||||||
out.println(node.dumpDatanode());
|
for (DatanodeDescriptor node : sortedDatanodeMap.values()) {
|
||||||
}
|
out.println(node.dumpDatanode());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -567,8 +563,8 @@ public class DatanodeManager {
|
||||||
* Remove a datanode
|
* Remove a datanode
|
||||||
* @throws UnregisteredNodeException
|
* @throws UnregisteredNodeException
|
||||||
*/
|
*/
|
||||||
public void removeDatanode(final DatanodeID node
|
public void removeDatanode(final DatanodeID node)
|
||||||
) throws UnregisteredNodeException {
|
throws UnregisteredNodeException {
|
||||||
namesystem.writeLock();
|
namesystem.writeLock();
|
||||||
try {
|
try {
|
||||||
final DatanodeDescriptor descriptor = getDatanode(node);
|
final DatanodeDescriptor descriptor = getDatanode(node);
|
||||||
|
@ -585,19 +581,17 @@ public class DatanodeManager {
|
||||||
|
|
||||||
/** Remove a dead datanode. */
|
/** Remove a dead datanode. */
|
||||||
void removeDeadDatanode(final DatanodeID nodeID) {
|
void removeDeadDatanode(final DatanodeID nodeID) {
|
||||||
synchronized(datanodeMap) {
|
DatanodeDescriptor d;
|
||||||
DatanodeDescriptor d;
|
try {
|
||||||
try {
|
d = getDatanode(nodeID);
|
||||||
d = getDatanode(nodeID);
|
} catch(IOException e) {
|
||||||
} catch(IOException e) {
|
d = null;
|
||||||
d = null;
|
}
|
||||||
}
|
if (d != null && isDatanodeDead(d)) {
|
||||||
if (d != null && isDatanodeDead(d)) {
|
NameNode.stateChangeLog.info(
|
||||||
NameNode.stateChangeLog.info(
|
"BLOCK* removeDeadDatanode: lost heartbeat from " + d);
|
||||||
"BLOCK* removeDeadDatanode: lost heartbeat from " + d);
|
removeDatanode(d);
|
||||||
removeDatanode(d);
|
}
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Is the datanode dead? */
|
/** Is the datanode dead? */
|
||||||
|
@ -611,14 +605,13 @@ public class DatanodeManager {
|
||||||
// To keep host2DatanodeMap consistent with datanodeMap,
|
// To keep host2DatanodeMap consistent with datanodeMap,
|
||||||
// remove from host2DatanodeMap the datanodeDescriptor removed
|
// remove from host2DatanodeMap the datanodeDescriptor removed
|
||||||
// from datanodeMap before adding node to host2DatanodeMap.
|
// from datanodeMap before adding node to host2DatanodeMap.
|
||||||
synchronized(datanodeMap) {
|
synchronized(this) {
|
||||||
host2DatanodeMap.remove(datanodeMap.put(node.getDatanodeUuid(), node));
|
host2DatanodeMap.remove(datanodeMap.put(node.getDatanodeUuid(), node));
|
||||||
}
|
}
|
||||||
|
|
||||||
networktopology.add(node); // may throw InvalidTopologyException
|
networktopology.add(node); // may throw InvalidTopologyException
|
||||||
host2DatanodeMap.add(node);
|
host2DatanodeMap.add(node);
|
||||||
checkIfClusterIsNowMultiRack(node);
|
checkIfClusterIsNowMultiRack(node);
|
||||||
blockManager.getBlockReportLeaseManager().register(node);
|
|
||||||
|
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug(getClass().getSimpleName() + ".addDatanode: "
|
LOG.debug(getClass().getSimpleName() + ".addDatanode: "
|
||||||
|
@ -629,11 +622,9 @@ public class DatanodeManager {
|
||||||
/** Physically remove node from datanodeMap. */
|
/** Physically remove node from datanodeMap. */
|
||||||
private void wipeDatanode(final DatanodeID node) {
|
private void wipeDatanode(final DatanodeID node) {
|
||||||
final String key = node.getDatanodeUuid();
|
final String key = node.getDatanodeUuid();
|
||||||
synchronized (datanodeMap) {
|
synchronized (this) {
|
||||||
host2DatanodeMap.remove(datanodeMap.remove(key));
|
host2DatanodeMap.remove(datanodeMap.remove(key));
|
||||||
}
|
}
|
||||||
// Also remove all block invalidation tasks under this node
|
|
||||||
blockManager.removeFromInvalidates(new DatanodeInfo(node));
|
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug(getClass().getSimpleName() + ".wipeDatanode("
|
LOG.debug(getClass().getSimpleName() + ".wipeDatanode("
|
||||||
+ node + "): storage " + key
|
+ node + "): storage " + key
|
||||||
|
@ -645,7 +636,7 @@ public class DatanodeManager {
|
||||||
if (version == null) {
|
if (version == null) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
synchronized(datanodeMap) {
|
synchronized(this) {
|
||||||
Integer count = this.datanodesSoftwareVersions.get(version);
|
Integer count = this.datanodesSoftwareVersions.get(version);
|
||||||
count = count == null ? 1 : count + 1;
|
count = count == null ? 1 : count + 1;
|
||||||
this.datanodesSoftwareVersions.put(version, count);
|
this.datanodesSoftwareVersions.put(version, count);
|
||||||
|
@ -656,7 +647,7 @@ public class DatanodeManager {
|
||||||
if (version == null) {
|
if (version == null) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
synchronized(datanodeMap) {
|
synchronized(this) {
|
||||||
Integer count = this.datanodesSoftwareVersions.get(version);
|
Integer count = this.datanodesSoftwareVersions.get(version);
|
||||||
if(count != null) {
|
if(count != null) {
|
||||||
if(count > 1) {
|
if(count > 1) {
|
||||||
|
@ -674,24 +665,22 @@ public class DatanodeManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void countSoftwareVersions() {
|
private void countSoftwareVersions() {
|
||||||
synchronized(datanodeMap) {
|
synchronized(this) {
|
||||||
HashMap<String, Integer> versionCount = new HashMap<>();
|
datanodesSoftwareVersions.clear();
|
||||||
for(DatanodeDescriptor dn: datanodeMap.values()) {
|
for(DatanodeDescriptor dn: datanodeMap.values()) {
|
||||||
// Check isAlive too because right after removeDatanode(),
|
// Check isAlive too because right after removeDatanode(),
|
||||||
// isDatanodeDead() is still true
|
// isDatanodeDead() is still true
|
||||||
if(shouldCountVersion(dn))
|
if (shouldCountVersion(dn)) {
|
||||||
{
|
Integer num = datanodesSoftwareVersions.get(dn.getSoftwareVersion());
|
||||||
Integer num = versionCount.get(dn.getSoftwareVersion());
|
|
||||||
num = num == null ? 1 : num+1;
|
num = num == null ? 1 : num+1;
|
||||||
versionCount.put(dn.getSoftwareVersion(), num);
|
datanodesSoftwareVersions.put(dn.getSoftwareVersion(), num);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
this.datanodesSoftwareVersions = versionCount;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public HashMap<String, Integer> getDatanodesSoftwareVersions() {
|
public HashMap<String, Integer> getDatanodesSoftwareVersions() {
|
||||||
synchronized(datanodeMap) {
|
synchronized(this) {
|
||||||
return new HashMap<> (this.datanodesSoftwareVersions);
|
return new HashMap<> (this.datanodesSoftwareVersions);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -747,13 +736,11 @@ public class DatanodeManager {
|
||||||
/**
|
/**
|
||||||
* Resolve network locations for specified hosts
|
* Resolve network locations for specified hosts
|
||||||
*
|
*
|
||||||
* @param names
|
|
||||||
* @return Network locations if available, Else returns null
|
* @return Network locations if available, Else returns null
|
||||||
*/
|
*/
|
||||||
public List<String> resolveNetworkLocation(List<String> names) {
|
public List<String> resolveNetworkLocation(List<String> names) {
|
||||||
// resolve its network location
|
// resolve its network location
|
||||||
List<String> rName = dnsToSwitchMapping.resolve(names);
|
return dnsToSwitchMapping.resolve(names);
|
||||||
return rName;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -807,10 +794,9 @@ public class DatanodeManager {
|
||||||
* This is used to not to display a decommissioned datanode to the operators.
|
* This is used to not to display a decommissioned datanode to the operators.
|
||||||
* @param nodeList , array list of live or dead nodes.
|
* @param nodeList , array list of live or dead nodes.
|
||||||
*/
|
*/
|
||||||
private void removeDecomNodeFromList(
|
private static void removeDecomNodeFromList(
|
||||||
final List<DatanodeDescriptor> nodeList) {
|
final List<DatanodeDescriptor> nodeList) {
|
||||||
Iterator<DatanodeDescriptor> it=null;
|
for (Iterator<DatanodeDescriptor> it = nodeList.iterator(); it.hasNext();) {
|
||||||
for (it = nodeList.iterator(); it.hasNext();) {
|
|
||||||
DatanodeDescriptor node = it.next();
|
DatanodeDescriptor node = it.next();
|
||||||
if (node.isDecommissioned()) {
|
if (node.isDecommissioned()) {
|
||||||
it.remove();
|
it.remove();
|
||||||
|
@ -968,6 +954,7 @@ public class DatanodeManager {
|
||||||
|
|
||||||
// register new datanode
|
// register new datanode
|
||||||
addDatanode(nodeDescr);
|
addDatanode(nodeDescr);
|
||||||
|
blockManager.getBlockReportLeaseManager().register(nodeDescr);
|
||||||
// also treat the registration message as a heartbeat
|
// also treat the registration message as a heartbeat
|
||||||
// no need to update its timestamp
|
// no need to update its timestamp
|
||||||
// because its is done when the descriptor is created
|
// because its is done when the descriptor is created
|
||||||
|
@ -1030,7 +1017,11 @@ public class DatanodeManager {
|
||||||
* 4. Removed from exclude --> stop decommission.
|
* 4. Removed from exclude --> stop decommission.
|
||||||
*/
|
*/
|
||||||
private void refreshDatanodes() {
|
private void refreshDatanodes() {
|
||||||
for(DatanodeDescriptor node : datanodeMap.values()) {
|
final Map<String, DatanodeDescriptor> copy;
|
||||||
|
synchronized (this) {
|
||||||
|
copy = new HashMap<>(datanodeMap);
|
||||||
|
}
|
||||||
|
for (DatanodeDescriptor node : copy.values()) {
|
||||||
// Check if not include.
|
// Check if not include.
|
||||||
if (!hostFileManager.isIncluded(node)) {
|
if (!hostFileManager.isIncluded(node)) {
|
||||||
node.setDisallowed(true); // case 2.
|
node.setDisallowed(true); // case 2.
|
||||||
|
@ -1047,7 +1038,7 @@ public class DatanodeManager {
|
||||||
/** @return the number of live datanodes. */
|
/** @return the number of live datanodes. */
|
||||||
public int getNumLiveDataNodes() {
|
public int getNumLiveDataNodes() {
|
||||||
int numLive = 0;
|
int numLive = 0;
|
||||||
synchronized (datanodeMap) {
|
synchronized (this) {
|
||||||
for(DatanodeDescriptor dn : datanodeMap.values()) {
|
for(DatanodeDescriptor dn : datanodeMap.values()) {
|
||||||
if (!isDatanodeDead(dn) ) {
|
if (!isDatanodeDead(dn) ) {
|
||||||
numLive++;
|
numLive++;
|
||||||
|
@ -1252,7 +1243,7 @@ public class DatanodeManager {
|
||||||
final HostFileManager.HostSet includedNodes = hostFileManager.getIncludes();
|
final HostFileManager.HostSet includedNodes = hostFileManager.getIncludes();
|
||||||
final HostFileManager.HostSet excludedNodes = hostFileManager.getExcludes();
|
final HostFileManager.HostSet excludedNodes = hostFileManager.getExcludes();
|
||||||
|
|
||||||
synchronized(datanodeMap) {
|
synchronized(this) {
|
||||||
nodes = new ArrayList<>(datanodeMap.size());
|
nodes = new ArrayList<>(datanodeMap.size());
|
||||||
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
||||||
final boolean isDead = isDatanodeDead(dn);
|
final boolean isDead = isDatanodeDead(dn);
|
||||||
|
@ -1327,155 +1318,160 @@ public class DatanodeManager {
|
||||||
node.setLastUpdateMonotonic(0);
|
node.setLastUpdateMonotonic(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private BlockRecoveryCommand getBlockRecoveryCommand(String blockPoolId,
|
||||||
|
DatanodeDescriptor nodeinfo) {
|
||||||
|
BlockInfo[] blocks = nodeinfo.getLeaseRecoveryCommand(Integer.MAX_VALUE);
|
||||||
|
if (blocks == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
BlockRecoveryCommand brCommand = new BlockRecoveryCommand(blocks.length);
|
||||||
|
for (BlockInfo b : blocks) {
|
||||||
|
BlockUnderConstructionFeature uc = b.getUnderConstructionFeature();
|
||||||
|
assert uc != null;
|
||||||
|
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
|
||||||
|
// Skip stale nodes during recovery
|
||||||
|
final List<DatanodeStorageInfo> recoveryLocations =
|
||||||
|
new ArrayList<>(storages.length);
|
||||||
|
for (DatanodeStorageInfo storage : storages) {
|
||||||
|
if (!storage.getDatanodeDescriptor().isStale(staleInterval)) {
|
||||||
|
recoveryLocations.add(storage);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// If we are performing a truncate recovery than set recovery fields
|
||||||
|
// to old block.
|
||||||
|
boolean truncateRecovery = uc.getTruncateBlock() != null;
|
||||||
|
boolean copyOnTruncateRecovery = truncateRecovery &&
|
||||||
|
uc.getTruncateBlock().getBlockId() != b.getBlockId();
|
||||||
|
ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ?
|
||||||
|
new ExtendedBlock(blockPoolId, uc.getTruncateBlock()) :
|
||||||
|
new ExtendedBlock(blockPoolId, b);
|
||||||
|
// If we only get 1 replica after eliminating stale nodes, choose all
|
||||||
|
// replicas for recovery and let the primary data node handle failures.
|
||||||
|
DatanodeInfo[] recoveryInfos;
|
||||||
|
if (recoveryLocations.size() > 1) {
|
||||||
|
if (recoveryLocations.size() != storages.length) {
|
||||||
|
LOG.info("Skipped stale nodes for recovery : "
|
||||||
|
+ (storages.length - recoveryLocations.size()));
|
||||||
|
}
|
||||||
|
recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(recoveryLocations);
|
||||||
|
} else {
|
||||||
|
// If too many replicas are stale, then choose all replicas to
|
||||||
|
// participate in block recovery.
|
||||||
|
recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages);
|
||||||
|
}
|
||||||
|
RecoveringBlock rBlock;
|
||||||
|
if (truncateRecovery) {
|
||||||
|
Block recoveryBlock = (copyOnTruncateRecovery) ? b : uc.getTruncateBlock();
|
||||||
|
rBlock = new RecoveringBlock(primaryBlock, recoveryInfos, recoveryBlock);
|
||||||
|
} else {
|
||||||
|
rBlock = new RecoveringBlock(primaryBlock, recoveryInfos,
|
||||||
|
uc.getBlockRecoveryId());
|
||||||
|
}
|
||||||
|
brCommand.add(rBlock);
|
||||||
|
}
|
||||||
|
return brCommand;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void addCacheCommands(String blockPoolId, DatanodeDescriptor nodeinfo,
|
||||||
|
List<DatanodeCommand> cmds) {
|
||||||
|
boolean sendingCachingCommands = false;
|
||||||
|
final long nowMs = monotonicNow();
|
||||||
|
if (shouldSendCachingCommands &&
|
||||||
|
((nowMs - nodeinfo.getLastCachingDirectiveSentTimeMs()) >=
|
||||||
|
timeBetweenResendingCachingDirectivesMs)) {
|
||||||
|
DatanodeCommand pendingCacheCommand = getCacheCommand(
|
||||||
|
nodeinfo.getPendingCached(), DatanodeProtocol.DNA_CACHE,
|
||||||
|
blockPoolId);
|
||||||
|
if (pendingCacheCommand != null) {
|
||||||
|
cmds.add(pendingCacheCommand);
|
||||||
|
sendingCachingCommands = true;
|
||||||
|
}
|
||||||
|
DatanodeCommand pendingUncacheCommand = getCacheCommand(
|
||||||
|
nodeinfo.getPendingUncached(), DatanodeProtocol.DNA_UNCACHE,
|
||||||
|
blockPoolId);
|
||||||
|
if (pendingUncacheCommand != null) {
|
||||||
|
cmds.add(pendingUncacheCommand);
|
||||||
|
sendingCachingCommands = true;
|
||||||
|
}
|
||||||
|
if (sendingCachingCommands) {
|
||||||
|
nodeinfo.setLastCachingDirectiveSentTimeMs(nowMs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/** Handle heartbeat from datanodes. */
|
/** Handle heartbeat from datanodes. */
|
||||||
public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
|
public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
|
||||||
StorageReport[] reports, final String blockPoolId,
|
StorageReport[] reports, final String blockPoolId,
|
||||||
long cacheCapacity, long cacheUsed, int xceiverCount,
|
long cacheCapacity, long cacheUsed, int xceiverCount,
|
||||||
int maxTransfers, int failedVolumes,
|
int maxTransfers, int failedVolumes,
|
||||||
VolumeFailureSummary volumeFailureSummary) throws IOException {
|
VolumeFailureSummary volumeFailureSummary) throws IOException {
|
||||||
synchronized (heartbeatManager) {
|
final DatanodeDescriptor nodeinfo;
|
||||||
synchronized (datanodeMap) {
|
try {
|
||||||
DatanodeDescriptor nodeinfo;
|
nodeinfo = getDatanode(nodeReg);
|
||||||
try {
|
} catch (UnregisteredNodeException e) {
|
||||||
nodeinfo = getDatanode(nodeReg);
|
return new DatanodeCommand[]{RegisterCommand.REGISTER};
|
||||||
} catch(UnregisteredNodeException e) {
|
}
|
||||||
return new DatanodeCommand[]{RegisterCommand.REGISTER};
|
|
||||||
}
|
|
||||||
|
|
||||||
// Check if this datanode should actually be shutdown instead.
|
|
||||||
if (nodeinfo != null && nodeinfo.isDisallowed()) {
|
|
||||||
setDatanodeDead(nodeinfo);
|
|
||||||
throw new DisallowedDatanodeException(nodeinfo);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (nodeinfo == null || !nodeinfo.isAlive()) {
|
// Check if this datanode should actually be shutdown instead.
|
||||||
return new DatanodeCommand[]{RegisterCommand.REGISTER};
|
if (nodeinfo != null && nodeinfo.isDisallowed()) {
|
||||||
}
|
setDatanodeDead(nodeinfo);
|
||||||
|
throw new DisallowedDatanodeException(nodeinfo);
|
||||||
|
}
|
||||||
|
|
||||||
heartbeatManager.updateHeartbeat(nodeinfo, reports,
|
if (nodeinfo == null || !nodeinfo.isAlive()) {
|
||||||
cacheCapacity, cacheUsed,
|
return new DatanodeCommand[]{RegisterCommand.REGISTER};
|
||||||
xceiverCount, failedVolumes,
|
}
|
||||||
volumeFailureSummary);
|
heartbeatManager.updateHeartbeat(nodeinfo, reports, cacheCapacity,
|
||||||
|
cacheUsed, xceiverCount, failedVolumes, volumeFailureSummary);
|
||||||
|
|
||||||
// If we are in safemode, do not send back any recovery / replication
|
// If we are in safemode, do not send back any recovery / replication
|
||||||
// requests. Don't even drain the existing queue of work.
|
// requests. Don't even drain the existing queue of work.
|
||||||
if(namesystem.isInSafeMode()) {
|
if (namesystem.isInSafeMode()) {
|
||||||
return new DatanodeCommand[0];
|
return new DatanodeCommand[0];
|
||||||
}
|
}
|
||||||
|
|
||||||
//check lease recovery
|
// block recovery command
|
||||||
BlockInfo[] blocks = nodeinfo.getLeaseRecoveryCommand(Integer.MAX_VALUE);
|
final BlockRecoveryCommand brCommand = getBlockRecoveryCommand(blockPoolId,
|
||||||
if (blocks != null) {
|
nodeinfo);
|
||||||
BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
|
if (brCommand != null) {
|
||||||
blocks.length);
|
return new DatanodeCommand[]{brCommand};
|
||||||
for (BlockInfo b : blocks) {
|
}
|
||||||
BlockUnderConstructionFeature uc = b.getUnderConstructionFeature();
|
|
||||||
assert uc != null;
|
|
||||||
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
|
|
||||||
// Skip stale nodes during recovery - not heart beated for some time (30s by default).
|
|
||||||
final List<DatanodeStorageInfo> recoveryLocations =
|
|
||||||
new ArrayList<>(storages.length);
|
|
||||||
for (DatanodeStorageInfo storage : storages) {
|
|
||||||
if (!storage.getDatanodeDescriptor().isStale(staleInterval)) {
|
|
||||||
recoveryLocations.add(storage);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// If we are performing a truncate recovery than set recovery fields
|
|
||||||
// to old block.
|
|
||||||
boolean truncateRecovery = uc.getTruncateBlock() != null;
|
|
||||||
boolean copyOnTruncateRecovery = truncateRecovery &&
|
|
||||||
uc.getTruncateBlock().getBlockId() != b.getBlockId();
|
|
||||||
ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ?
|
|
||||||
new ExtendedBlock(blockPoolId, uc.getTruncateBlock()) :
|
|
||||||
new ExtendedBlock(blockPoolId, b);
|
|
||||||
// If we only get 1 replica after eliminating stale nodes, then choose all
|
|
||||||
// replicas for recovery and let the primary data node handle failures.
|
|
||||||
DatanodeInfo[] recoveryInfos;
|
|
||||||
if (recoveryLocations.size() > 1) {
|
|
||||||
if (recoveryLocations.size() != storages.length) {
|
|
||||||
LOG.info("Skipped stale nodes for recovery : " +
|
|
||||||
(storages.length - recoveryLocations.size()));
|
|
||||||
}
|
|
||||||
recoveryInfos =
|
|
||||||
DatanodeStorageInfo.toDatanodeInfos(recoveryLocations);
|
|
||||||
} else {
|
|
||||||
// If too many replicas are stale, then choose all replicas to participate
|
|
||||||
// in block recovery.
|
|
||||||
recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages);
|
|
||||||
}
|
|
||||||
RecoveringBlock rBlock;
|
|
||||||
if(truncateRecovery) {
|
|
||||||
Block recoveryBlock = (copyOnTruncateRecovery) ? b :
|
|
||||||
uc.getTruncateBlock();
|
|
||||||
rBlock = new RecoveringBlock(primaryBlock, recoveryInfos,
|
|
||||||
recoveryBlock);
|
|
||||||
} else {
|
|
||||||
rBlock = new RecoveringBlock(primaryBlock, recoveryInfos,
|
|
||||||
uc.getBlockRecoveryId());
|
|
||||||
}
|
|
||||||
brCommand.add(rBlock);
|
|
||||||
}
|
|
||||||
return new DatanodeCommand[] { brCommand };
|
|
||||||
}
|
|
||||||
|
|
||||||
final List<DatanodeCommand> cmds = new ArrayList<>();
|
final List<DatanodeCommand> cmds = new ArrayList<>();
|
||||||
//check pending replication
|
// check pending replication
|
||||||
List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
|
List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
|
||||||
maxTransfers);
|
maxTransfers);
|
||||||
if (pendingList != null) {
|
if (pendingList != null) {
|
||||||
cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId,
|
cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId,
|
||||||
pendingList));
|
pendingList));
|
||||||
}
|
}
|
||||||
// checking pending erasure coding tasks
|
// check pending erasure coding tasks
|
||||||
List<BlockECRecoveryInfo> pendingECList =
|
List<BlockECRecoveryInfo> pendingECList = nodeinfo.getErasureCodeCommand(
|
||||||
nodeinfo.getErasureCodeCommand(maxTransfers);
|
maxTransfers);
|
||||||
if (pendingECList != null) {
|
if (pendingECList != null) {
|
||||||
cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY,
|
cmds.add(new BlockECRecoveryCommand(DNA_ERASURE_CODING_RECOVERY,
|
||||||
pendingECList));
|
pendingECList));
|
||||||
}
|
}
|
||||||
//check block invalidation
|
// check block invalidation
|
||||||
Block[] blks = nodeinfo.getInvalidateBlocks(blockInvalidateLimit);
|
Block[] blks = nodeinfo.getInvalidateBlocks(blockInvalidateLimit);
|
||||||
if (blks != null) {
|
if (blks != null) {
|
||||||
cmds.add(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE,
|
cmds.add(new BlockCommand(DatanodeProtocol.DNA_INVALIDATE, blockPoolId,
|
||||||
blockPoolId, blks));
|
blks));
|
||||||
}
|
}
|
||||||
boolean sendingCachingCommands = false;
|
// cache commands
|
||||||
long nowMs = monotonicNow();
|
addCacheCommands(blockPoolId, nodeinfo, cmds);
|
||||||
if (shouldSendCachingCommands &&
|
// key update command
|
||||||
((nowMs - nodeinfo.getLastCachingDirectiveSentTimeMs()) >=
|
blockManager.addKeyUpdateCommand(cmds, nodeinfo);
|
||||||
timeBetweenResendingCachingDirectivesMs)) {
|
|
||||||
DatanodeCommand pendingCacheCommand =
|
|
||||||
getCacheCommand(nodeinfo.getPendingCached(), nodeinfo,
|
|
||||||
DatanodeProtocol.DNA_CACHE, blockPoolId);
|
|
||||||
if (pendingCacheCommand != null) {
|
|
||||||
cmds.add(pendingCacheCommand);
|
|
||||||
sendingCachingCommands = true;
|
|
||||||
}
|
|
||||||
DatanodeCommand pendingUncacheCommand =
|
|
||||||
getCacheCommand(nodeinfo.getPendingUncached(), nodeinfo,
|
|
||||||
DatanodeProtocol.DNA_UNCACHE, blockPoolId);
|
|
||||||
if (pendingUncacheCommand != null) {
|
|
||||||
cmds.add(pendingUncacheCommand);
|
|
||||||
sendingCachingCommands = true;
|
|
||||||
}
|
|
||||||
if (sendingCachingCommands) {
|
|
||||||
nodeinfo.setLastCachingDirectiveSentTimeMs(nowMs);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
blockManager.addKeyUpdateCommand(cmds, nodeinfo);
|
// check for balancer bandwidth update
|
||||||
|
if (nodeinfo.getBalancerBandwidth() > 0) {
|
||||||
|
cmds.add(new BalancerBandwidthCommand(nodeinfo.getBalancerBandwidth()));
|
||||||
|
// set back to 0 to indicate that datanode has been sent the new value
|
||||||
|
nodeinfo.setBalancerBandwidth(0);
|
||||||
|
}
|
||||||
|
|
||||||
// check for balancer bandwidth update
|
if (!cmds.isEmpty()) {
|
||||||
if (nodeinfo.getBalancerBandwidth() > 0) {
|
return cmds.toArray(new DatanodeCommand[cmds.size()]);
|
||||||
cmds.add(new BalancerBandwidthCommand(nodeinfo.getBalancerBandwidth()));
|
|
||||||
// set back to 0 to indicate that datanode has been sent the new value
|
|
||||||
nodeinfo.setBalancerBandwidth(0);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!cmds.isEmpty()) {
|
|
||||||
return cmds.toArray(new DatanodeCommand[cmds.size()]);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return new DatanodeCommand[0];
|
return new DatanodeCommand[0];
|
||||||
|
@ -1486,14 +1482,13 @@ public class DatanodeManager {
|
||||||
*
|
*
|
||||||
* @param list The {@link CachedBlocksList}. This function
|
* @param list The {@link CachedBlocksList}. This function
|
||||||
* clears the list.
|
* clears the list.
|
||||||
* @param datanode The datanode.
|
|
||||||
* @param action The action to perform in the command.
|
* @param action The action to perform in the command.
|
||||||
* @param poolId The block pool id.
|
* @param poolId The block pool id.
|
||||||
* @return A DatanodeCommand to be sent back to the DN, or null if
|
* @return A DatanodeCommand to be sent back to the DN, or null if
|
||||||
* there is nothing to be done.
|
* there is nothing to be done.
|
||||||
*/
|
*/
|
||||||
private DatanodeCommand getCacheCommand(CachedBlocksList list,
|
private DatanodeCommand getCacheCommand(CachedBlocksList list, int action,
|
||||||
DatanodeDescriptor datanode, int action, String poolId) {
|
String poolId) {
|
||||||
int length = list.size();
|
int length = list.size();
|
||||||
if (length == 0) {
|
if (length == 0) {
|
||||||
return null;
|
return null;
|
||||||
|
@ -1501,9 +1496,7 @@ public class DatanodeManager {
|
||||||
// Read the existing cache commands.
|
// Read the existing cache commands.
|
||||||
long[] blockIds = new long[length];
|
long[] blockIds = new long[length];
|
||||||
int i = 0;
|
int i = 0;
|
||||||
for (Iterator<CachedBlock> iter = list.iterator();
|
for (CachedBlock cachedBlock : list) {
|
||||||
iter.hasNext(); ) {
|
|
||||||
CachedBlock cachedBlock = iter.next();
|
|
||||||
blockIds[i++] = cachedBlock.getBlockId();
|
blockIds[i++] = cachedBlock.getBlockId();
|
||||||
}
|
}
|
||||||
return new BlockIdCommand(action, poolId, blockIds);
|
return new BlockIdCommand(action, poolId, blockIds);
|
||||||
|
@ -1524,7 +1517,7 @@ public class DatanodeManager {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public void setBalancerBandwidth(long bandwidth) throws IOException {
|
public void setBalancerBandwidth(long bandwidth) throws IOException {
|
||||||
synchronized(datanodeMap) {
|
synchronized(this) {
|
||||||
for (DatanodeDescriptor nodeInfo : datanodeMap.values()) {
|
for (DatanodeDescriptor nodeInfo : datanodeMap.values()) {
|
||||||
nodeInfo.setBalancerBandwidth(bandwidth);
|
nodeInfo.setBalancerBandwidth(bandwidth);
|
||||||
}
|
}
|
||||||
|
@ -1533,7 +1526,7 @@ public class DatanodeManager {
|
||||||
|
|
||||||
public void markAllDatanodesStale() {
|
public void markAllDatanodesStale() {
|
||||||
LOG.info("Marking all datandoes as stale");
|
LOG.info("Marking all datandoes as stale");
|
||||||
synchronized (datanodeMap) {
|
synchronized (this) {
|
||||||
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
||||||
for(DatanodeStorageInfo storage : dn.getStorageInfos()) {
|
for(DatanodeStorageInfo storage : dn.getStorageInfos()) {
|
||||||
storage.markStaleAfterFailover();
|
storage.markStaleAfterFailover();
|
||||||
|
@ -1548,7 +1541,7 @@ public class DatanodeManager {
|
||||||
* recoveries, and replication requests.
|
* recoveries, and replication requests.
|
||||||
*/
|
*/
|
||||||
public void clearPendingQueues() {
|
public void clearPendingQueues() {
|
||||||
synchronized (datanodeMap) {
|
synchronized (this) {
|
||||||
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
||||||
dn.clearBlockQueues();
|
dn.clearBlockQueues();
|
||||||
}
|
}
|
||||||
|
@ -1560,7 +1553,7 @@ public class DatanodeManager {
|
||||||
* know about.
|
* know about.
|
||||||
*/
|
*/
|
||||||
public void resetLastCachingDirectiveSentTime() {
|
public void resetLastCachingDirectiveSentTime() {
|
||||||
synchronized (datanodeMap) {
|
synchronized (this) {
|
||||||
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
||||||
dn.setLastCachingDirectiveSentTimeMs(0L);
|
dn.setLastCachingDirectiveSentTimeMs(0L);
|
||||||
}
|
}
|
||||||
|
@ -1573,9 +1566,11 @@ public class DatanodeManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void clearPendingCachingCommands() {
|
public void clearPendingCachingCommands() {
|
||||||
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
synchronized (this) {
|
||||||
dn.getPendingCached().clear();
|
for (DatanodeDescriptor dn : datanodeMap.values()) {
|
||||||
dn.getPendingUncached().clear();
|
dn.getPendingCached().clear();
|
||||||
|
dn.getPendingUncached().clear();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -27,6 +27,8 @@ import org.apache.hadoop.fs.StorageType;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||||
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
|
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
|
||||||
|
import org.apache.hadoop.hdfs.server.protocol.RegisterCommand;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
||||||
import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
|
import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
|
||||||
import org.apache.hadoop.util.Daemon;
|
import org.apache.hadoop.util.Daemon;
|
||||||
|
|
|
@ -418,9 +418,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
*/
|
*/
|
||||||
private volatile boolean needRollbackFsImage;
|
private volatile boolean needRollbackFsImage;
|
||||||
|
|
||||||
// Block pool ID used by this namenode
|
|
||||||
private String blockPoolId;
|
|
||||||
|
|
||||||
final LeaseManager leaseManager = new LeaseManager(this);
|
final LeaseManager leaseManager = new LeaseManager(this);
|
||||||
|
|
||||||
Daemon nnrmthread = null; // NamenodeResourceMonitor thread
|
Daemon nnrmthread = null; // NamenodeResourceMonitor thread
|
||||||
|
@ -2348,12 +2345,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
}
|
}
|
||||||
|
|
||||||
ExtendedBlock getExtendedBlock(Block blk) {
|
ExtendedBlock getExtendedBlock(Block blk) {
|
||||||
return new ExtendedBlock(blockPoolId, blk);
|
return new ExtendedBlock(getBlockPoolId(), blk);
|
||||||
}
|
}
|
||||||
|
|
||||||
void setBlockPoolId(String bpid) {
|
void setBlockPoolId(String bpid) {
|
||||||
blockPoolId = bpid;
|
blockManager.setBlockPoolId(bpid);
|
||||||
blockManager.setBlockPoolId(blockPoolId);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -3489,11 +3485,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
* The given node has reported in. This method should:
|
* The given node has reported in. This method should:
|
||||||
* 1) Record the heartbeat, so the datanode isn't timed out
|
* 1) Record the heartbeat, so the datanode isn't timed out
|
||||||
* 2) Adjust usage stats for future block allocation
|
* 2) Adjust usage stats for future block allocation
|
||||||
*
|
*
|
||||||
* If a substantial amount of time passed since the last datanode
|
* If a substantial amount of time passed since the last datanode
|
||||||
* heartbeat then request an immediate block report.
|
* heartbeat then request an immediate block report.
|
||||||
*
|
*
|
||||||
* @return an array of datanode commands
|
* @return an array of datanode commands
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
|
HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
|
||||||
|
@ -3507,7 +3503,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
final int maxTransfer = blockManager.getMaxReplicationStreams()
|
final int maxTransfer = blockManager.getMaxReplicationStreams()
|
||||||
- xmitsInProgress;
|
- xmitsInProgress;
|
||||||
DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
|
DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
|
||||||
nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
|
nodeReg, reports, getBlockPoolId(), cacheCapacity, cacheUsed,
|
||||||
xceiverCount, maxTransfer, failedVolumes, volumeFailureSummary);
|
xceiverCount, maxTransfer, failedVolumes, volumeFailureSummary);
|
||||||
long blockReportLeaseId = 0;
|
long blockReportLeaseId = 0;
|
||||||
if (requestFullBlockReportLease) {
|
if (requestFullBlockReportLease) {
|
||||||
|
@ -5371,7 +5367,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
|
|
||||||
@Override // NameNodeMXBean
|
@Override // NameNodeMXBean
|
||||||
public String getBlockPoolId() {
|
public String getBlockPoolId() {
|
||||||
return blockPoolId;
|
return getBlockManager().getBlockPoolId();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override // NameNodeMXBean
|
@Override // NameNodeMXBean
|
||||||
|
@ -5960,7 +5956,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
||||||
}
|
}
|
||||||
|
|
||||||
void setRollingUpgradeInfo(boolean createdRollbackImages, long startTime) {
|
void setRollingUpgradeInfo(boolean createdRollbackImages, long startTime) {
|
||||||
rollingUpgradeInfo = new RollingUpgradeInfo(blockPoolId,
|
rollingUpgradeInfo = new RollingUpgradeInfo(getBlockPoolId(),
|
||||||
createdRollbackImages, startTime, 0L);
|
createdRollbackImages, startTime, 0L);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue