HDFS-9223. Code cleanup for DatanodeDescriptor and HeartbeatManager. Contributed by Jing Zhao.
This commit is contained in:
parent
a8070259f8
commit
be7a0add8b
|
@ -1527,6 +1527,8 @@ Release 2.8.0 - UNRELEASED
|
|||
HDFS-9238. Update TestFileCreation.testLeaseExpireHardLimit() to avoid using
|
||||
DataNodeTestUtils.getFile(). (Tony Wu via lei)
|
||||
|
||||
HDFS-9223. Code cleanup for DatanodeDescriptor and HeartbeatManager. (jing9)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
|
||||
|
|
|
@ -1022,9 +1022,9 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
void addKeyUpdateCommand(final List<DatanodeCommand> cmds,
|
||||
final DatanodeDescriptor nodeinfo) {
|
||||
// check access key update
|
||||
if (isBlockTokenEnabled() && nodeinfo.needKeyUpdate) {
|
||||
if (isBlockTokenEnabled() && nodeinfo.needKeyUpdate()) {
|
||||
cmds.add(new KeyUpdateCommand(blockTokenSecretManager.exportKeys()));
|
||||
nodeinfo.needKeyUpdate = false;
|
||||
nodeinfo.setNeedKeyUpdate(false);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1966,7 +1966,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
|
||||
try {
|
||||
node = datanodeManager.getDatanode(nodeID);
|
||||
if (node == null || !node.isAlive) {
|
||||
if (node == null || !node.isAlive()) {
|
||||
throw new IOException(
|
||||
"ProcessReport from dead or unregistered node: " + nodeID);
|
||||
}
|
||||
|
@ -3528,7 +3528,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
int deleted = 0;
|
||||
int receiving = 0;
|
||||
final DatanodeDescriptor node = datanodeManager.getDatanode(nodeID);
|
||||
if (node == null || !node.isAlive) {
|
||||
if (node == null || !node.isAlive()) {
|
||||
blockLog.warn("BLOCK* processIncrementalBlockReport"
|
||||
+ " is received from dead or unregistered node {}", nodeID);
|
||||
throw new IOException(
|
||||
|
@ -3678,7 +3678,7 @@ public class BlockManager implements BlockStatsMXBean {
|
|||
return false;
|
||||
}
|
||||
|
||||
if (node.isAlive) {
|
||||
if (node.isAlive()) {
|
||||
return true;
|
||||
}
|
||||
|
||||
|
|
|
@ -131,7 +131,7 @@ class BlocksMap {
|
|||
for(int idx = size - 1; idx >= 0; idx--) {
|
||||
DatanodeDescriptor dn = blockInfo.getDatanode(idx);
|
||||
if (dn != null) {
|
||||
dn.removeBlock(blockInfo); // remove from the list and wipe the location
|
||||
removeBlock(dn, blockInfo); // remove from the list and wipe the location
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -195,7 +195,7 @@ class BlocksMap {
|
|||
return false;
|
||||
|
||||
// remove block from the data-node list and the node from the block info
|
||||
boolean removed = node.removeBlock(info);
|
||||
boolean removed = removeBlock(node, info);
|
||||
|
||||
if (info.hasNoStorage() // no datanodes left
|
||||
&& info.isDeleted()) { // does not belong to a file
|
||||
|
@ -204,6 +204,16 @@ class BlocksMap {
|
|||
return removed;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove block from the list of blocks belonging to the data-node. Remove
|
||||
* data-node from the block.
|
||||
*/
|
||||
static boolean removeBlock(DatanodeDescriptor dn, BlockInfo b) {
|
||||
final DatanodeStorageInfo s = b.findStorageInfo(dn);
|
||||
// if block exists on this datanode
|
||||
return s != null && s.removeBlock(b);
|
||||
}
|
||||
|
||||
int size() {
|
||||
if (blocks != null) {
|
||||
return blocks.size();
|
||||
|
|
|
@ -67,29 +67,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
public static final Logger LOG =
|
||||
LoggerFactory.getLogger(DatanodeDescriptor.class);
|
||||
public static final DatanodeDescriptor[] EMPTY_ARRAY = {};
|
||||
|
||||
// Stores status of decommissioning.
|
||||
// If node is not decommissioning, do not use this object for anything.
|
||||
public final DecommissioningStatus decommissioningStatus =
|
||||
new DecommissioningStatus();
|
||||
|
||||
private long curBlockReportId = 0;
|
||||
|
||||
private BitSet curBlockReportRpcsSeen = null;
|
||||
|
||||
public int updateBlockReportContext(BlockReportContext context) {
|
||||
if (curBlockReportId != context.getReportId()) {
|
||||
curBlockReportId = context.getReportId();
|
||||
curBlockReportRpcsSeen = new BitSet(context.getTotalRpcs());
|
||||
}
|
||||
curBlockReportRpcsSeen.set(context.getCurRpc());
|
||||
return curBlockReportRpcsSeen.cardinality();
|
||||
}
|
||||
|
||||
public void clearBlockReportContext() {
|
||||
curBlockReportId = 0;
|
||||
curBlockReportRpcsSeen = null;
|
||||
}
|
||||
private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
|
||||
private static final List<DatanodeStorageInfo> EMPTY_STORAGE_INFO_LIST =
|
||||
ImmutableList.of();
|
||||
|
||||
/** Block and targets pair */
|
||||
@InterfaceAudience.Private
|
||||
|
@ -106,7 +86,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
|
||||
/** A BlockTargetPair queue. */
|
||||
private static class BlockQueue<E> {
|
||||
private final Queue<E> blockq = new LinkedList<E>();
|
||||
private final Queue<E> blockq = new LinkedList<>();
|
||||
|
||||
/** Size of the queue */
|
||||
synchronized int size() {return blockq.size();}
|
||||
|
@ -132,7 +112,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
/**
|
||||
* Returns <tt>true</tt> if the queue contains the specified element.
|
||||
*/
|
||||
boolean contains(E e) {
|
||||
synchronized boolean contains(E e) {
|
||||
return blockq.contains(e);
|
||||
}
|
||||
|
||||
|
@ -141,9 +121,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
}
|
||||
}
|
||||
|
||||
private final Map<String, DatanodeStorageInfo> storageMap =
|
||||
new HashMap<>();
|
||||
|
||||
/**
|
||||
* A list of CachedBlock objects on this datanode.
|
||||
*/
|
||||
|
@ -172,6 +149,18 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
}
|
||||
}
|
||||
|
||||
// Stores status of decommissioning.
|
||||
// If node is not decommissioning, do not use this object for anything.
|
||||
public final DecommissioningStatus decommissioningStatus =
|
||||
new DecommissioningStatus();
|
||||
|
||||
private long curBlockReportId = 0;
|
||||
|
||||
private BitSet curBlockReportRpcsSeen = null;
|
||||
|
||||
private final Map<String, DatanodeStorageInfo> storageMap =
|
||||
new HashMap<>();
|
||||
|
||||
/**
|
||||
* The blocks which we want to cache on this DataNode.
|
||||
*/
|
||||
|
@ -191,18 +180,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
private final CachedBlocksList pendingUncached =
|
||||
new CachedBlocksList(this, CachedBlocksList.Type.PENDING_UNCACHED);
|
||||
|
||||
public CachedBlocksList getPendingCached() {
|
||||
return pendingCached;
|
||||
}
|
||||
|
||||
public CachedBlocksList getCached() {
|
||||
return cached;
|
||||
}
|
||||
|
||||
public CachedBlocksList getPendingUncached() {
|
||||
return pendingUncached;
|
||||
}
|
||||
|
||||
/**
|
||||
* The time when the last batch of caching directives was sent, in
|
||||
* monotonic milliseconds.
|
||||
|
@ -211,9 +188,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
|
||||
// isAlive == heartbeats.contains(this)
|
||||
// This is an optimization, because contains takes O(n) time on Arraylist
|
||||
public boolean isAlive = false;
|
||||
public boolean needKeyUpdate = false;
|
||||
|
||||
private boolean isAlive = false;
|
||||
private boolean needKeyUpdate = false;
|
||||
|
||||
// A system administrator can tune the balancer bandwidth parameter
|
||||
// (dfs.balance.bandwidthPerSec) dynamically by calling
|
||||
|
@ -245,7 +221,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
private EnumCounters<StorageType> prevApproxBlocksScheduled
|
||||
= new EnumCounters<>(StorageType.class);
|
||||
private long lastBlocksScheduledRollTime = 0;
|
||||
private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
|
||||
private int volumeFailures = 0;
|
||||
private VolumeFailureSummary volumeFailureSummary = null;
|
||||
|
||||
|
@ -281,6 +256,48 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
updateHeartbeatState(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0, null);
|
||||
}
|
||||
|
||||
public int updateBlockReportContext(BlockReportContext context) {
|
||||
if (curBlockReportId != context.getReportId()) {
|
||||
curBlockReportId = context.getReportId();
|
||||
curBlockReportRpcsSeen = new BitSet(context.getTotalRpcs());
|
||||
}
|
||||
curBlockReportRpcsSeen.set(context.getCurRpc());
|
||||
return curBlockReportRpcsSeen.cardinality();
|
||||
}
|
||||
|
||||
public void clearBlockReportContext() {
|
||||
curBlockReportId = 0;
|
||||
curBlockReportRpcsSeen = null;
|
||||
}
|
||||
|
||||
public CachedBlocksList getPendingCached() {
|
||||
return pendingCached;
|
||||
}
|
||||
|
||||
public CachedBlocksList getCached() {
|
||||
return cached;
|
||||
}
|
||||
|
||||
public CachedBlocksList getPendingUncached() {
|
||||
return pendingUncached;
|
||||
}
|
||||
|
||||
public boolean isAlive() {
|
||||
return isAlive;
|
||||
}
|
||||
|
||||
public void setAlive(boolean isAlive) {
|
||||
this.isAlive = isAlive;
|
||||
}
|
||||
|
||||
public boolean needKeyUpdate() {
|
||||
return needKeyUpdate;
|
||||
}
|
||||
|
||||
public void setNeedKeyUpdate(boolean needKeyUpdate) {
|
||||
this.needKeyUpdate = needKeyUpdate;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public DatanodeStorageInfo getStorageInfo(String storageID) {
|
||||
synchronized (storageMap) {
|
||||
|
@ -316,9 +333,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
}
|
||||
}
|
||||
|
||||
static final private List<DatanodeStorageInfo> EMPTY_STORAGE_INFO_LIST =
|
||||
ImmutableList.of();
|
||||
|
||||
List<DatanodeStorageInfo> removeZombieStorages() {
|
||||
List<DatanodeStorageInfo> zombies = null;
|
||||
synchronized (storageMap) {
|
||||
|
@ -344,28 +358,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
return zombies == null ? EMPTY_STORAGE_INFO_LIST : zombies;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove block from the list of blocks belonging to the data-node. Remove
|
||||
* data-node from the block.
|
||||
*/
|
||||
boolean removeBlock(BlockInfo b) {
|
||||
final DatanodeStorageInfo s = b.findStorageInfo(this);
|
||||
// if block exists on this datanode
|
||||
if (s != null) {
|
||||
return s.removeBlock(b);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove block from the list of blocks belonging to the data-node. Remove
|
||||
* data-node from the block.
|
||||
*/
|
||||
boolean removeBlock(String storageID, BlockInfo b) {
|
||||
DatanodeStorageInfo s = getStorageInfo(storageID);
|
||||
return s != null && s.removeBlock(b);
|
||||
}
|
||||
|
||||
public void resetBlocks() {
|
||||
setCapacity(0);
|
||||
setRemaining(0);
|
||||
|
@ -384,10 +376,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
public void clearBlockQueues() {
|
||||
synchronized (invalidateBlocks) {
|
||||
this.invalidateBlocks.clear();
|
||||
this.recoverBlocks.clear();
|
||||
this.replicateBlocks.clear();
|
||||
this.erasurecodeBlocks.clear();
|
||||
}
|
||||
this.recoverBlocks.clear();
|
||||
this.replicateBlocks.clear();
|
||||
this.erasurecodeBlocks.clear();
|
||||
// pendingCached, cached, and pendingUncached are protected by the
|
||||
// FSN lock.
|
||||
this.pendingCached.clear();
|
||||
|
@ -589,10 +581,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
return new BlockIterator(getStorageInfos());
|
||||
}
|
||||
|
||||
Iterator<BlockInfo> getBlockIterator(final String storageID) {
|
||||
return new BlockIterator(getStorageInfo(storageID));
|
||||
}
|
||||
|
||||
void incrementPendingReplicationWithoutTargets() {
|
||||
pendingReplicationWithoutTargets++;
|
||||
}
|
||||
|
@ -662,16 +650,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
return erasurecodeBlocks.size();
|
||||
}
|
||||
|
||||
/**
|
||||
* The number of block invalidation items that are pending to
|
||||
* be sent to the datanode
|
||||
*/
|
||||
int getNumberOfBlocksToBeInvalidated() {
|
||||
synchronized (invalidateBlocks) {
|
||||
return invalidateBlocks.size();
|
||||
}
|
||||
}
|
||||
|
||||
public List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
|
||||
return replicateBlocks.poll(maxTransfers);
|
||||
}
|
||||
|
@ -714,7 +692,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
*
|
||||
* @param t requested storage type
|
||||
* @param blockSize requested block size
|
||||
* @return
|
||||
*/
|
||||
public DatanodeStorageInfo chooseStorage4Block(StorageType t,
|
||||
long blockSize) {
|
||||
|
@ -724,8 +701,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
long remaining = 0;
|
||||
DatanodeStorageInfo storage = null;
|
||||
for (DatanodeStorageInfo s : getStorageInfos()) {
|
||||
if (s.getState() == State.NORMAL &&
|
||||
s.getStorageType() == t) {
|
||||
if (s.getState() == State.NORMAL && s.getStorageType() == t) {
|
||||
if (storage == null) {
|
||||
storage = s;
|
||||
}
|
||||
|
@ -761,7 +737,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
|
||||
/** Increment the number of blocks scheduled. */
|
||||
void incrementBlocksScheduled(StorageType t) {
|
||||
currApproxBlocksScheduled.add(t, 1);;
|
||||
currApproxBlocksScheduled.add(t, 1);
|
||||
}
|
||||
|
||||
/** Decrement the number of blocks scheduled. */
|
||||
|
@ -805,7 +781,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
|
||||
synchronized void set(int underRep,
|
||||
int onlyRep, int underConstruction) {
|
||||
if (isDecommissionInProgress() == false) {
|
||||
if (!isDecommissionInProgress()) {
|
||||
return;
|
||||
}
|
||||
underReplicatedBlocks = underRep;
|
||||
|
@ -815,21 +791,21 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
|
||||
/** @return the number of under-replicated blocks */
|
||||
public synchronized int getUnderReplicatedBlocks() {
|
||||
if (isDecommissionInProgress() == false) {
|
||||
if (!isDecommissionInProgress()) {
|
||||
return 0;
|
||||
}
|
||||
return underReplicatedBlocks;
|
||||
}
|
||||
/** @return the number of decommission-only replicas */
|
||||
public synchronized int getDecommissionOnlyReplicas() {
|
||||
if (isDecommissionInProgress() == false) {
|
||||
if (!isDecommissionInProgress()) {
|
||||
return 0;
|
||||
}
|
||||
return decommissionOnlyReplicas;
|
||||
}
|
||||
/** @return the number of under-replicated blocks in open files */
|
||||
public synchronized int getUnderReplicatedInOpenFiles() {
|
||||
if (isDecommissionInProgress() == false) {
|
||||
if (!isDecommissionInProgress()) {
|
||||
return 0;
|
||||
}
|
||||
return underReplicatedInOpenFiles;
|
||||
|
@ -840,7 +816,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
}
|
||||
/** @return start time */
|
||||
public synchronized long getStartTime() {
|
||||
if (isDecommissionInProgress() == false) {
|
||||
if (!isDecommissionInProgress()) {
|
||||
return 0;
|
||||
}
|
||||
return startTime;
|
||||
|
@ -962,8 +938,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
|
|||
}
|
||||
|
||||
/**
|
||||
* checks whether atleast first block report has been received
|
||||
* @return
|
||||
* @return whether at least first block report has been received
|
||||
*/
|
||||
public boolean checkBlockReportReceived() {
|
||||
if(this.getStorageInfos().length == 0) {
|
||||
|
|
|
@ -309,7 +309,7 @@ public class DatanodeManager {
|
|||
|
||||
void activate(final Configuration conf) {
|
||||
decomManager.activate(conf);
|
||||
heartbeatManager.activate(conf);
|
||||
heartbeatManager.activate();
|
||||
}
|
||||
|
||||
void close() {
|
||||
|
@ -659,7 +659,7 @@ public class DatanodeManager {
|
|||
}
|
||||
|
||||
private boolean shouldCountVersion(DatanodeDescriptor node) {
|
||||
return node.getSoftwareVersion() != null && node.isAlive &&
|
||||
return node.getSoftwareVersion() != null && node.isAlive() &&
|
||||
!isDatanodeDead(node);
|
||||
}
|
||||
|
||||
|
@ -1343,7 +1343,7 @@ public class DatanodeManager {
|
|||
throw new DisallowedDatanodeException(nodeinfo);
|
||||
}
|
||||
|
||||
if (nodeinfo == null || !nodeinfo.isAlive) {
|
||||
if (nodeinfo == null || !nodeinfo.isAlive()) {
|
||||
return new DatanodeCommand[]{RegisterCommand.REGISTER};
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,193 @@
|
|||
package org.apache.hadoop.hdfs.server.blockmanagement;
|
||||
|
||||
import org.apache.hadoop.fs.StorageType;
|
||||
import org.apache.hadoop.hdfs.DFSUtilClient;
|
||||
|
||||
import java.util.EnumMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Datanode statistics.
|
||||
* For decommissioning/decommissioned nodes, only used capacity is counted.
|
||||
*/
|
||||
class DatanodeStats {
|
||||
|
||||
private final StorageTypeStatsMap statsMap = new StorageTypeStatsMap();
|
||||
private long capacityTotal = 0L;
|
||||
private long capacityUsed = 0L;
|
||||
private long capacityRemaining = 0L;
|
||||
private long blockPoolUsed = 0L;
|
||||
private int xceiverCount = 0;
|
||||
private long cacheCapacity = 0L;
|
||||
private long cacheUsed = 0L;
|
||||
|
||||
private int nodesInService = 0;
|
||||
private int nodesInServiceXceiverCount = 0;
|
||||
private int expiredHeartbeats = 0;
|
||||
|
||||
synchronized void add(final DatanodeDescriptor node) {
|
||||
capacityUsed += node.getDfsUsed();
|
||||
blockPoolUsed += node.getBlockPoolUsed();
|
||||
xceiverCount += node.getXceiverCount();
|
||||
if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
|
||||
nodesInService++;
|
||||
nodesInServiceXceiverCount += node.getXceiverCount();
|
||||
capacityTotal += node.getCapacity();
|
||||
capacityRemaining += node.getRemaining();
|
||||
} else {
|
||||
capacityTotal += node.getDfsUsed();
|
||||
}
|
||||
cacheCapacity += node.getCacheCapacity();
|
||||
cacheUsed += node.getCacheUsed();
|
||||
Set<StorageType> storageTypes = new HashSet<>();
|
||||
for (DatanodeStorageInfo storageInfo : node.getStorageInfos()) {
|
||||
statsMap.addStorage(storageInfo, node);
|
||||
storageTypes.add(storageInfo.getStorageType());
|
||||
}
|
||||
for (StorageType storageType : storageTypes) {
|
||||
statsMap.addNode(storageType, node);
|
||||
}
|
||||
}
|
||||
|
||||
synchronized void subtract(final DatanodeDescriptor node) {
|
||||
capacityUsed -= node.getDfsUsed();
|
||||
blockPoolUsed -= node.getBlockPoolUsed();
|
||||
xceiverCount -= node.getXceiverCount();
|
||||
if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
|
||||
nodesInService--;
|
||||
nodesInServiceXceiverCount -= node.getXceiverCount();
|
||||
capacityTotal -= node.getCapacity();
|
||||
capacityRemaining -= node.getRemaining();
|
||||
} else {
|
||||
capacityTotal -= node.getDfsUsed();
|
||||
}
|
||||
cacheCapacity -= node.getCacheCapacity();
|
||||
cacheUsed -= node.getCacheUsed();
|
||||
Set<StorageType> storageTypes = new HashSet<>();
|
||||
for (DatanodeStorageInfo storageInfo : node.getStorageInfos()) {
|
||||
statsMap.subtractStorage(storageInfo, node);
|
||||
storageTypes.add(storageInfo.getStorageType());
|
||||
}
|
||||
for (StorageType storageType : storageTypes) {
|
||||
statsMap.subtractNode(storageType, node);
|
||||
}
|
||||
}
|
||||
|
||||
/** Increment expired heartbeat counter. */
|
||||
void incrExpiredHeartbeats() {
|
||||
expiredHeartbeats++;
|
||||
}
|
||||
|
||||
synchronized Map<StorageType, StorageTypeStats> getStatsMap() {
|
||||
return statsMap.get();
|
||||
}
|
||||
|
||||
synchronized long getCapacityTotal() {
|
||||
return capacityTotal;
|
||||
}
|
||||
|
||||
synchronized long getCapacityUsed() {
|
||||
return capacityUsed;
|
||||
}
|
||||
|
||||
synchronized long getCapacityRemaining() {
|
||||
return capacityRemaining;
|
||||
}
|
||||
|
||||
synchronized long getBlockPoolUsed() {
|
||||
return blockPoolUsed;
|
||||
}
|
||||
|
||||
synchronized int getXceiverCount() {
|
||||
return xceiverCount;
|
||||
}
|
||||
|
||||
synchronized long getCacheCapacity() {
|
||||
return cacheCapacity;
|
||||
}
|
||||
|
||||
synchronized long getCacheUsed() {
|
||||
return cacheUsed;
|
||||
}
|
||||
|
||||
synchronized int getNodesInService() {
|
||||
return nodesInService;
|
||||
}
|
||||
|
||||
synchronized int getNodesInServiceXceiverCount() {
|
||||
return nodesInServiceXceiverCount;
|
||||
}
|
||||
|
||||
synchronized int getExpiredHeartbeats() {
|
||||
return expiredHeartbeats;
|
||||
}
|
||||
|
||||
synchronized float getCapacityRemainingPercent() {
|
||||
return DFSUtilClient.getPercentRemaining(capacityRemaining, capacityTotal);
|
||||
}
|
||||
|
||||
synchronized float getPercentBlockPoolUsed() {
|
||||
return DFSUtilClient.getPercentUsed(blockPoolUsed, capacityTotal);
|
||||
}
|
||||
|
||||
synchronized long getCapacityUsedNonDFS() {
|
||||
final long nonDFSUsed = capacityTotal - capacityRemaining - capacityUsed;
|
||||
return nonDFSUsed < 0L? 0L : nonDFSUsed;
|
||||
}
|
||||
|
||||
synchronized float getCapacityUsedPercent() {
|
||||
return DFSUtilClient.getPercentUsed(capacityUsed, capacityTotal);
|
||||
}
|
||||
|
||||
static final class StorageTypeStatsMap {
|
||||
|
||||
private Map<StorageType, StorageTypeStats> storageTypeStatsMap =
|
||||
new EnumMap<>(StorageType.class);
|
||||
|
||||
private Map<StorageType, StorageTypeStats> get() {
|
||||
return new EnumMap<>(storageTypeStatsMap);
|
||||
}
|
||||
|
||||
private void addNode(StorageType storageType,
|
||||
final DatanodeDescriptor node) {
|
||||
StorageTypeStats storageTypeStats =
|
||||
storageTypeStatsMap.get(storageType);
|
||||
if (storageTypeStats == null) {
|
||||
storageTypeStats = new StorageTypeStats();
|
||||
storageTypeStatsMap.put(storageType, storageTypeStats);
|
||||
}
|
||||
storageTypeStats.addNode(node);
|
||||
}
|
||||
|
||||
private void addStorage(final DatanodeStorageInfo info,
|
||||
final DatanodeDescriptor node) {
|
||||
StorageTypeStats storageTypeStats =
|
||||
storageTypeStatsMap.get(info.getStorageType());
|
||||
if (storageTypeStats == null) {
|
||||
storageTypeStats = new StorageTypeStats();
|
||||
storageTypeStatsMap.put(info.getStorageType(), storageTypeStats);
|
||||
}
|
||||
storageTypeStats.addStorage(info, node);
|
||||
}
|
||||
|
||||
private void subtractStorage(final DatanodeStorageInfo info,
|
||||
final DatanodeDescriptor node) {
|
||||
StorageTypeStats storageTypeStats =
|
||||
storageTypeStatsMap.get(info.getStorageType());
|
||||
if (storageTypeStats != null) {
|
||||
storageTypeStats.subtractStorage(info, node);
|
||||
}
|
||||
}
|
||||
|
||||
private void subtractNode(StorageType storageType,
|
||||
final DatanodeDescriptor node) {
|
||||
StorageTypeStats storageTypeStats =
|
||||
storageTypeStatsMap.get(storageType);
|
||||
if (storageTypeStats != null) {
|
||||
storageTypeStats.subtractNode(node);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -216,7 +216,7 @@ public class DecommissionManager {
|
|||
hbManager.stopDecommission(node);
|
||||
// Over-replicated blocks will be detected and processed when
|
||||
// the dead node comes back and send in its full block report.
|
||||
if (node.isAlive) {
|
||||
if (node.isAlive()) {
|
||||
blockManager.processOverReplicatedBlocksOnReCommission(node);
|
||||
}
|
||||
// Remove from tracking in DecommissionManager
|
||||
|
|
|
@ -18,18 +18,13 @@
|
|||
package org.apache.hadoop.hdfs.server.blockmanagement;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.StorageType;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSUtilClient;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
||||
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
|
||||
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
||||
|
@ -57,10 +52,10 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
* and removes them from the list.
|
||||
* It is synchronized by the heartbeat manager lock.
|
||||
*/
|
||||
private final List<DatanodeDescriptor> datanodes = new ArrayList<DatanodeDescriptor>();
|
||||
private final List<DatanodeDescriptor> datanodes = new ArrayList<>();
|
||||
|
||||
/** Statistics, which are synchronized by the heartbeat manager lock. */
|
||||
private final Stats stats = new Stats();
|
||||
private final DatanodeStats stats = new DatanodeStats();
|
||||
|
||||
/** The time period to check for expired datanodes */
|
||||
private final long heartbeatRecheckInterval;
|
||||
|
@ -96,7 +91,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
}
|
||||
}
|
||||
|
||||
void activate(Configuration conf) {
|
||||
void activate() {
|
||||
heartbeatThread.start();
|
||||
}
|
||||
|
||||
|
@ -105,7 +100,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
try {
|
||||
// This will no effect if the thread hasn't yet been started.
|
||||
heartbeatThread.join(3000);
|
||||
} catch (InterruptedException e) {
|
||||
} catch (InterruptedException ignored) {
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -114,74 +109,69 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized long getCapacityTotal() {
|
||||
return stats.capacityTotal;
|
||||
public long getCapacityTotal() {
|
||||
return stats.getCapacityTotal();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long getCapacityUsed() {
|
||||
return stats.capacityUsed;
|
||||
public long getCapacityUsed() {
|
||||
return stats.getCapacityUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized float getCapacityUsedPercent() {
|
||||
return DFSUtilClient.getPercentUsed(stats.capacityUsed, stats.capacityTotal);
|
||||
public float getCapacityUsedPercent() {
|
||||
return stats.getCapacityUsedPercent();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long getCapacityRemaining() {
|
||||
return stats.capacityRemaining;
|
||||
public long getCapacityRemaining() {
|
||||
return stats.getCapacityRemaining();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized float getCapacityRemainingPercent() {
|
||||
return DFSUtilClient.getPercentRemaining(stats.capacityRemaining,
|
||||
stats.capacityTotal);
|
||||
public float getCapacityRemainingPercent() {
|
||||
return stats.getCapacityRemainingPercent();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long getBlockPoolUsed() {
|
||||
return stats.blockPoolUsed;
|
||||
public long getBlockPoolUsed() {
|
||||
return stats.getBlockPoolUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized float getPercentBlockPoolUsed() {
|
||||
return DFSUtilClient.getPercentUsed(stats.blockPoolUsed,
|
||||
stats.capacityTotal);
|
||||
public float getPercentBlockPoolUsed() {
|
||||
return stats.getPercentBlockPoolUsed();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long getCapacityUsedNonDFS() {
|
||||
final long nonDFSUsed = stats.capacityTotal
|
||||
- stats.capacityRemaining - stats.capacityUsed;
|
||||
return nonDFSUsed < 0L? 0L : nonDFSUsed;
|
||||
public long getCapacityUsedNonDFS() {
|
||||
return stats.getCapacityUsedNonDFS();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int getXceiverCount() {
|
||||
return stats.xceiverCount;
|
||||
public int getXceiverCount() {
|
||||
return stats.getXceiverCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int getInServiceXceiverCount() {
|
||||
return stats.nodesInServiceXceiverCount;
|
||||
public int getInServiceXceiverCount() {
|
||||
return stats.getNodesInServiceXceiverCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int getNumDatanodesInService() {
|
||||
return stats.nodesInService;
|
||||
public int getNumDatanodesInService() {
|
||||
return stats.getNodesInService();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long getCacheCapacity() {
|
||||
return stats.cacheCapacity;
|
||||
public long getCacheCapacity() {
|
||||
return stats.getCacheCapacity();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized long getCacheUsed() {
|
||||
return stats.cacheUsed;
|
||||
public long getCacheUsed() {
|
||||
return stats.getCacheUsed();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public synchronized long[] getStats() {
|
||||
|
@ -195,17 +185,17 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
}
|
||||
|
||||
@Override
|
||||
public synchronized int getExpiredHeartbeats() {
|
||||
return stats.expiredHeartbeats;
|
||||
public int getExpiredHeartbeats() {
|
||||
return stats.getExpiredHeartbeats();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<StorageType, StorageTypeStats> getStorageTypeStats() {
|
||||
return stats.statsMap.get();
|
||||
public Map<StorageType, StorageTypeStats> getStorageTypeStats() {
|
||||
return stats.getStatsMap();
|
||||
}
|
||||
|
||||
synchronized void register(final DatanodeDescriptor d) {
|
||||
if (!d.isAlive) {
|
||||
if (!d.isAlive()) {
|
||||
addDatanode(d);
|
||||
|
||||
//update its timestamp
|
||||
|
@ -221,14 +211,14 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
// update in-service node count
|
||||
stats.add(d);
|
||||
datanodes.add(d);
|
||||
d.isAlive = true;
|
||||
d.setAlive(true);
|
||||
}
|
||||
|
||||
synchronized void removeDatanode(DatanodeDescriptor node) {
|
||||
if (node.isAlive) {
|
||||
if (node.isAlive()) {
|
||||
stats.subtract(node);
|
||||
datanodes.remove(node);
|
||||
node.isAlive = false;
|
||||
node.setAlive(false);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -243,7 +233,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
}
|
||||
|
||||
synchronized void startDecommission(final DatanodeDescriptor node) {
|
||||
if (!node.isAlive) {
|
||||
if (!node.isAlive()) {
|
||||
LOG.info("Dead node {} is decommissioned immediately.", node);
|
||||
node.setDecommissioned();
|
||||
} else {
|
||||
|
@ -255,8 +245,8 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
|
||||
synchronized void stopDecommission(final DatanodeDescriptor node) {
|
||||
LOG.info("Stopping decommissioning of {} node {}",
|
||||
node.isAlive ? "live" : "dead", node);
|
||||
if (!node.isAlive) {
|
||||
node.isAlive() ? "live" : "dead", node);
|
||||
if (!node.isAlive()) {
|
||||
node.stopDecommission();
|
||||
} else {
|
||||
stats.subtract(node);
|
||||
|
@ -302,6 +292,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
* B. Remove all blocks in PendingDataNodeMessages for the failed storage
|
||||
* when we remove all blocks from BlocksMap for that storage.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
void heartbeatCheck() {
|
||||
final DatanodeManager dm = blockManager.getDatanodeManager();
|
||||
// It's OK to check safe mode w/o taking the lock here, we re-check
|
||||
|
@ -354,16 +345,14 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
}
|
||||
|
||||
allAlive = dead == null && failedStorage == null;
|
||||
if (!allAlive && namesystem.isInStartupSafeMode()) {
|
||||
return;
|
||||
}
|
||||
if (dead != null) {
|
||||
// acquire the fsnamesystem lock, and then remove the dead node.
|
||||
namesystem.writeLock();
|
||||
try {
|
||||
if (namesystem.isInStartupSafeMode()) {
|
||||
return;
|
||||
}
|
||||
synchronized(this) {
|
||||
dm.removeDeadDatanode(dead);
|
||||
}
|
||||
dm.removeDeadDatanode(dead);
|
||||
} finally {
|
||||
namesystem.writeUnlock();
|
||||
}
|
||||
|
@ -372,12 +361,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
// acquire the fsnamesystem lock, and remove blocks on the storage.
|
||||
namesystem.writeLock();
|
||||
try {
|
||||
if (namesystem.isInStartupSafeMode()) {
|
||||
return;
|
||||
}
|
||||
synchronized(this) {
|
||||
blockManager.removeBlocksAssociatedTo(failedStorage);
|
||||
}
|
||||
blockManager.removeBlocksAssociatedTo(failedStorage);
|
||||
} finally {
|
||||
namesystem.writeUnlock();
|
||||
}
|
||||
|
@ -385,7 +369,6 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
/** Periodically check heartbeat and update block key */
|
||||
private class Monitor implements Runnable {
|
||||
private long lastHeartbeatCheck;
|
||||
|
@ -404,7 +387,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
if (blockManager.shouldUpdateBlockKey(now - lastBlockKeyUpdate)) {
|
||||
synchronized(HeartbeatManager.this) {
|
||||
for(DatanodeDescriptor d : datanodes) {
|
||||
d.needKeyUpdate = true;
|
||||
d.setNeedKeyUpdate(true);
|
||||
}
|
||||
}
|
||||
lastBlockKeyUpdate = now;
|
||||
|
@ -414,7 +397,7 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
}
|
||||
try {
|
||||
Thread.sleep(5000); // 5 seconds
|
||||
} catch (InterruptedException ie) {
|
||||
} catch (InterruptedException ignored) {
|
||||
}
|
||||
// avoid declaring nodes dead for another cycle if a GC pause lasts
|
||||
// longer than the node recheck interval
|
||||
|
@ -425,143 +408,4 @@ class HeartbeatManager implements DatanodeStatistics {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/** Datanode statistics.
|
||||
* For decommissioning/decommissioned nodes, only used capacity is counted.
|
||||
*/
|
||||
private static class Stats {
|
||||
|
||||
private final StorageTypeStatsMap statsMap = new StorageTypeStatsMap();
|
||||
|
||||
private long capacityTotal = 0L;
|
||||
private long capacityUsed = 0L;
|
||||
private long capacityRemaining = 0L;
|
||||
private long blockPoolUsed = 0L;
|
||||
private int xceiverCount = 0;
|
||||
private long cacheCapacity = 0L;
|
||||
private long cacheUsed = 0L;
|
||||
|
||||
private int nodesInService = 0;
|
||||
private int nodesInServiceXceiverCount = 0;
|
||||
|
||||
private int expiredHeartbeats = 0;
|
||||
|
||||
private void add(final DatanodeDescriptor node) {
|
||||
capacityUsed += node.getDfsUsed();
|
||||
blockPoolUsed += node.getBlockPoolUsed();
|
||||
xceiverCount += node.getXceiverCount();
|
||||
if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
|
||||
nodesInService++;
|
||||
nodesInServiceXceiverCount += node.getXceiverCount();
|
||||
capacityTotal += node.getCapacity();
|
||||
capacityRemaining += node.getRemaining();
|
||||
} else {
|
||||
capacityTotal += node.getDfsUsed();
|
||||
}
|
||||
cacheCapacity += node.getCacheCapacity();
|
||||
cacheUsed += node.getCacheUsed();
|
||||
Set<StorageType> storageTypes = new HashSet<>();
|
||||
for (DatanodeStorageInfo storageInfo : node.getStorageInfos()) {
|
||||
statsMap.addStorage(storageInfo, node);
|
||||
storageTypes.add(storageInfo.getStorageType());
|
||||
}
|
||||
for (StorageType storageType : storageTypes) {
|
||||
statsMap.addNode(storageType, node);
|
||||
}
|
||||
}
|
||||
|
||||
private void subtract(final DatanodeDescriptor node) {
|
||||
capacityUsed -= node.getDfsUsed();
|
||||
blockPoolUsed -= node.getBlockPoolUsed();
|
||||
xceiverCount -= node.getXceiverCount();
|
||||
if (!(node.isDecommissionInProgress() || node.isDecommissioned())) {
|
||||
nodesInService--;
|
||||
nodesInServiceXceiverCount -= node.getXceiverCount();
|
||||
capacityTotal -= node.getCapacity();
|
||||
capacityRemaining -= node.getRemaining();
|
||||
} else {
|
||||
capacityTotal -= node.getDfsUsed();
|
||||
}
|
||||
cacheCapacity -= node.getCacheCapacity();
|
||||
cacheUsed -= node.getCacheUsed();
|
||||
Set<StorageType> storageTypes = new HashSet<>();
|
||||
for (DatanodeStorageInfo storageInfo : node.getStorageInfos()) {
|
||||
statsMap.subtractStorage(storageInfo, node);
|
||||
storageTypes.add(storageInfo.getStorageType());
|
||||
}
|
||||
for (StorageType storageType : storageTypes) {
|
||||
statsMap.subtractNode(storageType, node);
|
||||
}
|
||||
}
|
||||
|
||||
/** Increment expired heartbeat counter. */
|
||||
private void incrExpiredHeartbeats() {
|
||||
expiredHeartbeats++;
|
||||
}
|
||||
}
|
||||
|
||||
/** StorageType specific statistics.
|
||||
* For decommissioning/decommissioned nodes, only used capacity is counted.
|
||||
*/
|
||||
|
||||
static final class StorageTypeStatsMap {
|
||||
|
||||
private Map<StorageType, StorageTypeStats> storageTypeStatsMap =
|
||||
new IdentityHashMap<>();
|
||||
|
||||
private StorageTypeStatsMap() {}
|
||||
|
||||
private StorageTypeStatsMap(StorageTypeStatsMap other) {
|
||||
storageTypeStatsMap =
|
||||
new IdentityHashMap<>(other.storageTypeStatsMap);
|
||||
for (Map.Entry<StorageType, StorageTypeStats> entry :
|
||||
storageTypeStatsMap.entrySet()) {
|
||||
entry.setValue(new StorageTypeStats(entry.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
private Map<StorageType, StorageTypeStats> get() {
|
||||
return Collections.unmodifiableMap(storageTypeStatsMap);
|
||||
}
|
||||
|
||||
private void addNode(StorageType storageType,
|
||||
final DatanodeDescriptor node) {
|
||||
StorageTypeStats storageTypeStats =
|
||||
storageTypeStatsMap.get(storageType);
|
||||
if (storageTypeStats == null) {
|
||||
storageTypeStats = new StorageTypeStats();
|
||||
storageTypeStatsMap.put(storageType, storageTypeStats);
|
||||
}
|
||||
storageTypeStats.addNode(node);
|
||||
}
|
||||
|
||||
private void addStorage(final DatanodeStorageInfo info,
|
||||
final DatanodeDescriptor node) {
|
||||
StorageTypeStats storageTypeStats =
|
||||
storageTypeStatsMap.get(info.getStorageType());
|
||||
if (storageTypeStats == null) {
|
||||
storageTypeStats = new StorageTypeStats();
|
||||
storageTypeStatsMap.put(info.getStorageType(), storageTypeStats);
|
||||
}
|
||||
storageTypeStats.addStorage(info, node);
|
||||
}
|
||||
|
||||
private void subtractStorage(final DatanodeStorageInfo info,
|
||||
final DatanodeDescriptor node) {
|
||||
StorageTypeStats storageTypeStats =
|
||||
storageTypeStatsMap.get(info.getStorageType());
|
||||
if (storageTypeStats != null) {
|
||||
storageTypeStats.subtractStorage(info, node);
|
||||
}
|
||||
}
|
||||
|
||||
private void subtractNode(StorageType storageType,
|
||||
final DatanodeDescriptor node) {
|
||||
StorageTypeStats storageTypeStats =
|
||||
storageTypeStatsMap.get(storageType);
|
||||
if (storageTypeStats != null) {
|
||||
storageTypeStats.subtractNode(node);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -86,7 +86,7 @@ class ReplicaUnderConstruction extends Block {
|
|||
* Is data-node the replica belongs to alive.
|
||||
*/
|
||||
boolean isAlive() {
|
||||
return expectedLocation.getDatanodeDescriptor().isAlive;
|
||||
return expectedLocation.getDatanodeDescriptor().isAlive();
|
||||
}
|
||||
|
||||
@Override // Block
|
||||
|
|
|
@ -934,7 +934,7 @@ public final class CacheManager {
|
|||
try {
|
||||
final DatanodeDescriptor datanode =
|
||||
blockManager.getDatanodeManager().getDatanode(datanodeID);
|
||||
if (datanode == null || !datanode.isAlive) {
|
||||
if (datanode == null || !datanode.isAlive()) {
|
||||
throw new IOException(
|
||||
"processCacheReport from dead or unregistered datanode: " +
|
||||
datanode);
|
||||
|
|
|
@ -1768,7 +1768,7 @@ public class DFSTestUtil {
|
|||
FSNamesystem namesystem = cluster.getNamesystem();
|
||||
final DatanodeDescriptor dd = BlockManagerTestUtil.getDatanode(
|
||||
namesystem, nodeID);
|
||||
return (dd.isAlive == alive);
|
||||
return (dd.isAlive() == alive);
|
||||
}
|
||||
}, 100, waitTime);
|
||||
}
|
||||
|
|
|
@ -595,7 +595,7 @@ public class TestBlockManager {
|
|||
public void testSafeModeIBR() throws Exception {
|
||||
DatanodeDescriptor node = spy(nodes.get(0));
|
||||
DatanodeStorageInfo ds = node.getStorageInfos()[0];
|
||||
node.isAlive = true;
|
||||
node.setAlive(true);
|
||||
|
||||
DatanodeRegistration nodeReg =
|
||||
new DatanodeRegistration(node, null, null, "");
|
||||
|
@ -640,7 +640,7 @@ public class TestBlockManager {
|
|||
DatanodeDescriptor node = spy(nodes.get(0));
|
||||
DatanodeStorageInfo ds = node.getStorageInfos()[0];
|
||||
|
||||
node.isAlive = true;
|
||||
node.setAlive(true);
|
||||
|
||||
DatanodeRegistration nodeReg =
|
||||
new DatanodeRegistration(node, null, null, "");
|
||||
|
@ -672,7 +672,7 @@ public class TestBlockManager {
|
|||
|
||||
DatanodeDescriptor node = nodes.get(0);
|
||||
DatanodeStorageInfo ds = node.getStorageInfos()[0];
|
||||
node.isAlive = true;
|
||||
node.setAlive(true);
|
||||
DatanodeRegistration nodeReg = new DatanodeRegistration(node, null, null, "");
|
||||
|
||||
// register new node
|
||||
|
|
|
@ -38,7 +38,9 @@ public class TestBlockUnderConstructionFeature {
|
|||
DatanodeStorageInfo s3 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.3", "s3");
|
||||
DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
|
||||
|
||||
dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
|
||||
dd1.setAlive(true);
|
||||
dd2.setAlive(true);
|
||||
dd3.setAlive(true);
|
||||
BlockInfoContiguous blockInfo = new BlockInfoContiguous(
|
||||
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3);
|
||||
blockInfo.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
|
||||
|
|
|
@ -66,7 +66,7 @@ public class TestDatanodeDescriptor {
|
|||
assertTrue(storages[0].addBlock(blk) == AddBlockResult.ADDED);
|
||||
assertEquals(1, dd.numBlocks());
|
||||
// remove a non-existent block
|
||||
assertFalse(dd.removeBlock(blk1));
|
||||
assertFalse(BlocksMap.removeBlock(dd, blk1));
|
||||
assertEquals(1, dd.numBlocks());
|
||||
// add an existent block
|
||||
assertFalse(storages[0].addBlock(blk) == AddBlockResult.ADDED);
|
||||
|
@ -75,10 +75,10 @@ public class TestDatanodeDescriptor {
|
|||
assertTrue(storages[0].addBlock(blk1) == AddBlockResult.ADDED);
|
||||
assertEquals(2, dd.numBlocks());
|
||||
// remove first block
|
||||
assertTrue(dd.removeBlock(blk));
|
||||
assertTrue(BlocksMap.removeBlock(dd, blk));
|
||||
assertEquals(1, dd.numBlocks());
|
||||
// remove second block
|
||||
assertTrue(dd.removeBlock(blk1));
|
||||
assertTrue(BlocksMap.removeBlock(dd, blk1));
|
||||
assertEquals(0, dd.numBlocks());
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue