Revert 1140913 and 1140909 for HDFS-2107.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1140920 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2011-06-29 00:26:41 +00:00
parent d58e3efe92
commit 97b6ca4dd7
53 changed files with 353 additions and 492 deletions

View File

@ -534,9 +534,6 @@ Trunk (unreleased changes)
HDFS-2110. StreamFile and ByteRangeInputStream cleanup. (eli)
HDFS-2107. Move block management code from o.a.h.h.s.namenode to a new
package o.a.h.h.s.blockmanagement. (szetszwo)
OPTIMIZATIONS
HDFS-1458. Improve checkpoint performance by avoiding unnecessary image

View File

@ -61,10 +61,10 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.namenode.BlockPlacementPolicyDefault;
import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
import org.apache.hadoop.io.IOUtils;

View File

@ -1,57 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
/**
* A immutable object that stores the number of live replicas and
* the number of decommissined Replicas.
*/
public class NumberReplicas {
private int liveReplicas;
private int decommissionedReplicas;
private int corruptReplicas;
private int excessReplicas;
NumberReplicas() {
initialize(0, 0, 0, 0);
}
NumberReplicas(int live, int decommissioned, int corrupt, int excess) {
initialize(live, decommissioned, corrupt, excess);
}
void initialize(int live, int decommissioned, int corrupt, int excess) {
liveReplicas = live;
decommissionedReplicas = decommissioned;
corruptReplicas = corrupt;
excessReplicas = excess;
}
public int liveReplicas() {
return liveReplicas;
}
public int decommissionedReplicas() {
return decommissionedReplicas;
}
public int corruptReplicas() {
return corruptReplicas;
}
public int excessReplicas() {
return excessReplicas;
}
}

View File

@ -26,8 +26,8 @@ import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.URL;
import java.net.URLEncoder;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
@ -45,13 +45,14 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.http.HtmlQuoting;
import org.apache.hadoop.io.Text;

View File

@ -15,17 +15,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.util.LightWeightGSet;
/**
* Internal class for block metadata.
*/
public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
private INodeFile inode;
/** For implementing {@link LightWeightGSet.LinkedElement} interface */
@ -45,12 +44,12 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* Construct an entry for blocksmap
* @param replication the block's replication factor
*/
public BlockInfo(int replication) {
protected BlockInfo(int replication) {
this.triplets = new Object[3*replication];
this.inode = null;
}
public BlockInfo(Block blk, int replication) {
protected BlockInfo(Block blk, int replication) {
super(blk);
this.triplets = new Object[3*replication];
this.inode = null;
@ -66,11 +65,11 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
this.inode = from.inode;
}
public INodeFile getINode() {
INodeFile getINode() {
return inode;
}
public void setINode(INodeFile inode) {
void setINode(INodeFile inode) {
this.inode = inode;
}
@ -163,7 +162,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
/**
* Add data-node this block belongs to.
*/
public boolean addNode(DatanodeDescriptor node) {
boolean addNode(DatanodeDescriptor node) {
if(findDatanode(node) >= 0) // the node is already there
return false;
// find the last null node
@ -177,7 +176,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
/**
* Remove data-node from the block.
*/
public boolean removeNode(DatanodeDescriptor node) {
boolean removeNode(DatanodeDescriptor node) {
int dnIndex = findDatanode(node);
if(dnIndex < 0) // the node is not found
return false;
@ -219,7 +218,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* If the head is null then form a new list.
* @return current block as the new head of the list.
*/
public BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
BlockInfo listInsert(BlockInfo head, DatanodeDescriptor dn) {
int dnIndex = this.findDatanode(dn);
assert dnIndex >= 0 : "Data node is not found: current";
assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
@ -239,7 +238,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* @return the new head of the list or null if the list becomes
* empty after deletion.
*/
public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
if(head == null)
return null;
int dnIndex = this.findDatanode(dn);
@ -285,7 +284,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* to {@link BlockInfoUnderConstruction}.
* @return {@link BlockUCState#COMPLETE}
*/
public BlockUCState getBlockUCState() {
BlockUCState getBlockUCState() {
return BlockUCState.COMPLETE;
}
@ -294,7 +293,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
*
* @return true if the state of the block is {@link BlockUCState#COMPLETE}
*/
public boolean isComplete() {
boolean isComplete() {
return getBlockUCState().equals(BlockUCState.COMPLETE);
}
@ -303,7 +302,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
*
* @return BlockInfoUnderConstruction - an under construction block.
*/
public BlockInfoUnderConstruction convertToBlockUnderConstruction(
BlockInfoUnderConstruction convertToBlockUnderConstruction(
BlockUCState s, DatanodeDescriptor[] targets) {
if(isComplete()) {
return new BlockInfoUnderConstruction(

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.util.ArrayList;
@ -24,13 +24,12 @@ import java.util.List;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
/**
* Represents a block that is currently being constructed.<br>
* This is usually the last block of a file opened for write or append.
*/
public class BlockInfoUnderConstruction extends BlockInfo {
class BlockInfoUnderConstruction extends BlockInfo {
/** Block state. See {@link BlockUCState} */
private BlockUCState blockUCState;
@ -129,14 +128,11 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* Create block and set its state to
* {@link BlockUCState#UNDER_CONSTRUCTION}.
*/
public BlockInfoUnderConstruction(Block blk, int replication) {
BlockInfoUnderConstruction(Block blk, int replication) {
this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
}
/**
* Create a block that is currently being constructed.
*/
public BlockInfoUnderConstruction(Block blk, int replication,
BlockInfoUnderConstruction(Block blk, int replication,
BlockUCState state,
DatanodeDescriptor[] targets) {
super(blk, replication);
@ -164,8 +160,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
return new BlockInfo(this);
}
/** Set expected locations */
public void setExpectedLocations(DatanodeDescriptor[] targets) {
void setExpectedLocations(DatanodeDescriptor[] targets) {
int numLocations = targets == null ? 0 : targets.length;
this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
for(int i = 0; i < numLocations; i++)
@ -177,7 +172,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* Create array of expected replica locations
* (as has been assigned by chooseTargets()).
*/
public DatanodeDescriptor[] getExpectedLocations() {
DatanodeDescriptor[] getExpectedLocations() {
int numLocations = replicas == null ? 0 : replicas.size();
DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations];
for(int i = 0; i < numLocations; i++)
@ -185,8 +180,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
return locations;
}
/** Get the number of expected locations */
public int getNumExpectedLocations() {
int getNumExpectedLocations() {
return replicas == null ? 0 : replicas.size();
}
@ -195,7 +189,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* @see BlockUCState
*/
@Override // BlockInfo
public BlockUCState getBlockUCState() {
BlockUCState getBlockUCState() {
return blockUCState;
}
@ -203,8 +197,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
blockUCState = s;
}
/** Get block recovery ID */
public long getBlockRecoveryId() {
long getBlockRecoveryId() {
return blockRecoveryId;
}
@ -227,7 +220,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* Find the first alive data-node starting from the previous primary and
* make it primary.
*/
public void initializeBlockRecovery(long recoveryId) {
void initializeBlockRecovery(long recoveryId) {
setBlockUCState(BlockUCState.UNDER_RECOVERY);
blockRecoveryId = recoveryId;
if (replicas.size() == 0) {

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.io.PrintWriter;
@ -39,14 +39,10 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks.BlockIterator;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks;
/**
* Keeps information related to the blocks stored in the Hadoop cluster.
@ -61,43 +57,18 @@ public class BlockManager {
private final FSNamesystem namesystem;
private volatile long pendingReplicationBlocksCount = 0L;
private volatile long corruptReplicaBlocksCount = 0L;
private volatile long underReplicatedBlocksCount = 0L;
public volatile long scheduledReplicationBlocksCount = 0L;
private volatile long excessBlocksCount = 0L;
private volatile long pendingDeletionBlocksCount = 0L;
volatile long pendingReplicationBlocksCount = 0L;
volatile long corruptReplicaBlocksCount = 0L;
volatile long underReplicatedBlocksCount = 0L;
volatile long scheduledReplicationBlocksCount = 0L;
volatile long excessBlocksCount = 0L;
volatile long pendingDeletionBlocksCount = 0L;
/** Used by metrics */
public long getPendingReplicationBlocksCount() {
return pendingReplicationBlocksCount;
}
/** Used by metrics */
public long getUnderReplicatedBlocksCount() {
return underReplicatedBlocksCount;
}
/** Used by metrics */
public long getCorruptReplicaBlocksCount() {
return corruptReplicaBlocksCount;
}
/** Used by metrics */
public long getScheduledReplicationBlocksCount() {
return scheduledReplicationBlocksCount;
}
/** Used by metrics */
public long getPendingDeletionBlocksCount() {
return pendingDeletionBlocksCount;
}
/** Used by metrics */
public long getExcessBlocksCount() {
return excessBlocksCount;
}
/**
* Mapping: Block -> { INode, datanodes, self ref }
* Updated only in response to client-sent information.
*/
public final BlocksMap blocksMap;
//
// Mapping: Block -> { INode, datanodes, self ref }
// Updated only in response to client-sent information.
//
final BlocksMap blocksMap;
//
// Store blocks-->datanodedescriptor(s) map of corrupt replicas
@ -119,24 +90,24 @@ public class BlockManager {
// eventually remove these extras.
// Mapping: StorageID -> TreeSet<Block>
//
public final Map<String, Collection<Block>> excessReplicateMap =
Map<String, Collection<Block>> excessReplicateMap =
new TreeMap<String, Collection<Block>>();
//
// Store set of Blocks that need to be replicated 1 or more times.
// We also store pending replication-orders.
//
public UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
UnderReplicatedBlocks neededReplications = new UnderReplicatedBlocks();
private PendingReplicationBlocks pendingReplications;
// The maximum number of replicas allowed for a block
public int maxReplication;
int maxReplication;
// How many outgoing replication streams a given node should have at one time
public int maxReplicationStreams;
int maxReplicationStreams;
// Minimum copies needed or else write is disallowed
public int minReplication;
int minReplication;
// Default number of replicas
public int defaultReplication;
int defaultReplication;
// How many entries are returned by getCorruptInodes()
int maxCorruptFilesReturned;
@ -150,9 +121,9 @@ public class BlockManager {
Random r = new Random();
// for block replicas placement
public BlockPlacementPolicy replicator;
BlockPlacementPolicy replicator;
public BlockManager(FSNamesystem fsn, Configuration conf) throws IOException {
BlockManager(FSNamesystem fsn, Configuration conf) throws IOException {
this(fsn, conf, DEFAULT_INITIAL_MAP_CAPACITY);
}
@ -207,16 +178,16 @@ public class BlockManager {
FSNamesystem.LOG.info("shouldCheckForEnoughRacks = " + shouldCheckForEnoughRacks);
}
public void activate() {
void activate() {
pendingReplications.start();
}
public void close() {
void close() {
if (pendingReplications != null) pendingReplications.stop();
blocksMap.close();
}
public void metaSave(PrintWriter out) {
void metaSave(PrintWriter out) {
//
// Dump contents of neededReplication
//
@ -278,7 +249,7 @@ public class BlockManager {
* @param block
* @return true if the block has minimum replicas
*/
public boolean checkMinReplication(Block block) {
boolean checkMinReplication(Block block) {
return (countNodes(block).liveReplicas() >= minReplication);
}
@ -326,7 +297,7 @@ public class BlockManager {
* @throws IOException if the block does not have at least a minimal number
* of replicas reported from data-nodes.
*/
public void commitOrCompleteLastBlock(INodeFileUnderConstruction fileINode,
void commitOrCompleteLastBlock(INodeFileUnderConstruction fileINode,
Block commitBlock) throws IOException {
if(commitBlock == null)
@ -391,7 +362,7 @@ public class BlockManager {
* @param fileINode file
* @return the last block locations if the block is partial or null otherwise
*/
public LocatedBlock convertLastBlockToUnderConstruction(
LocatedBlock convertLastBlockToUnderConstruction(
INodeFileUnderConstruction fileINode) throws IOException {
BlockInfo oldBlock = fileINode.getLastBlock();
if(oldBlock == null ||
@ -422,7 +393,7 @@ public class BlockManager {
/**
* Get all valid locations of the block
*/
public ArrayList<String> getValidLocations(Block block) {
ArrayList<String> getValidLocations(Block block) {
ArrayList<String> machineSet =
new ArrayList<String>(blocksMap.numNodes(block));
for(Iterator<DatanodeDescriptor> it =
@ -436,7 +407,7 @@ public class BlockManager {
return machineSet;
}
public List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset,
List<LocatedBlock> getBlockLocations(BlockInfo[] blocks, long offset,
long length, int nrBlocksToReturn) throws IOException {
int curBlk = 0;
long curPos = 0, blkSize = 0;
@ -465,15 +436,11 @@ public class BlockManager {
return results;
}
/** @return a LocatedBlock for the given block */
public LocatedBlock getBlockLocation(final BlockInfo blk, final long pos
/** @param needBlockToken
* @return a LocatedBlock for the given block */
LocatedBlock getBlockLocation(final BlockInfo blk, final long pos
) throws IOException {
if (blk instanceof BlockInfoUnderConstruction) {
if (blk.isComplete()) {
throw new IOException(
"blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
+ ", blk=" + blk);
}
if (!blk.isComplete()) {
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
final DatanodeDescriptor[] locations = uc.getExpectedLocations();
return namesystem.createLocatedBlock(uc, locations, pos, false);
@ -509,7 +476,7 @@ public class BlockManager {
* Check whether the replication parameter is within the range
* determined by system configuration.
*/
public void verifyReplication(String src,
void verifyReplication(String src,
short replication,
String clientName) throws IOException {
@ -577,7 +544,7 @@ public class BlockManager {
* @param b block
* @param dn datanode
*/
public void addToInvalidates(Block b, DatanodeInfo dn) {
void addToInvalidates(Block b, DatanodeInfo dn) {
addToInvalidates(b, dn, true);
}
@ -618,7 +585,7 @@ public class BlockManager {
}
}
public void findAndMarkBlockAsCorrupt(Block blk,
void findAndMarkBlockAsCorrupt(Block blk,
DatanodeInfo dn) throws IOException {
BlockInfo storedBlock = getStoredBlock(blk);
if (storedBlock == null) {
@ -701,14 +668,14 @@ public class BlockManager {
}
}
public void updateState() {
void updateState() {
pendingReplicationBlocksCount = pendingReplications.size();
underReplicatedBlocksCount = neededReplications.size();
corruptReplicaBlocksCount = corruptReplicas.size();
}
/** Return number of under-replicated but not missing blocks */
public int getUnderReplicatedNotMissingBlocks() {
int getUnderReplicatedNotMissingBlocks() {
return neededReplications.getUnderReplicatedBlockCount();
}
@ -717,7 +684,7 @@ public class BlockManager {
* @param nodesToProcess number of datanodes to schedule deletion work
* @return total number of block for deletion
*/
public int computeInvalidateWork(int nodesToProcess) {
int computeInvalidateWork(int nodesToProcess) {
int numOfNodes = recentInvalidateSets.size();
nodesToProcess = Math.min(numOfNodes, nodesToProcess);
@ -757,7 +724,7 @@ public class BlockManager {
*
* @return number of blocks scheduled for replication during this iteration.
*/
public int computeReplicationWork(int blocksToProcess) throws IOException {
int computeReplicationWork(int blocksToProcess) throws IOException {
// Choose the blocks to be replicated
List<List<Block>> blocksToReplicate =
chooseUnderReplicatedBlocks(blocksToProcess);
@ -1064,7 +1031,7 @@ public class BlockManager {
* If there were any replication requests that timed out, reap them
* and put them back into the neededReplication queue
*/
public void processPendingReplications() {
void processPendingReplications() {
Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
if (timedOutItems != null) {
namesystem.writeLock();
@ -1497,7 +1464,7 @@ public class BlockManager {
short fileReplication = fileINode.getReplication();
if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
neededReplications.remove(storedBlock, numCurrentReplica,
num.decommissionedReplicas(), fileReplication);
num.decommissionedReplicas, fileReplication);
} else {
updateNeededReplications(storedBlock, curReplicaDelta, 0);
}
@ -1558,7 +1525,7 @@ public class BlockManager {
* For each block in the name-node verify whether it belongs to any file,
* over or under replicated. Place it into the respective queue.
*/
public void processMisReplicatedBlocks() {
void processMisReplicatedBlocks() {
long nrInvalid = 0, nrOverReplicated = 0, nrUnderReplicated = 0;
namesystem.writeLock();
try {
@ -1603,7 +1570,7 @@ public class BlockManager {
* If there are any extras, call chooseExcessReplicates() to
* mark them in the excessReplicateMap.
*/
public void processOverReplicatedBlock(Block block, short replication,
void processOverReplicatedBlock(Block block, short replication,
DatanodeDescriptor addedNode, DatanodeDescriptor delNodeHint) {
assert namesystem.hasWriteLock();
if (addedNode == delNodeHint) {
@ -1630,7 +1597,7 @@ public class BlockManager {
addedNode, delNodeHint, replicator);
}
public void addToExcessReplicate(DatanodeInfo dn, Block block) {
void addToExcessReplicate(DatanodeInfo dn, Block block) {
assert namesystem.hasWriteLock();
Collection<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
if (excessBlocks == null) {
@ -1651,7 +1618,7 @@ public class BlockManager {
* Modify (block-->datanode) map. Possibly generate replication tasks, if the
* removed block is still valid.
*/
public void removeStoredBlock(Block block, DatanodeDescriptor node) {
void removeStoredBlock(Block block, DatanodeDescriptor node) {
if(NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
+ block + " from " + node.getName());
@ -1706,7 +1673,7 @@ public class BlockManager {
/**
* The given node is reporting that it received a certain block.
*/
public void addBlock(DatanodeDescriptor node, Block block, String delHint)
void addBlock(DatanodeDescriptor node, Block block, String delHint)
throws IOException {
// decrement number of blocks scheduled to this datanode.
node.decBlocksScheduled();
@ -1759,7 +1726,7 @@ public class BlockManager {
/**
* Return the number of nodes that are live and decommissioned.
*/
public NumberReplicas countNodes(Block b) {
NumberReplicas countNodes(Block b) {
int count = 0;
int live = 0;
int corrupt = 0;
@ -1838,7 +1805,7 @@ public class BlockManager {
* Return true if there are any blocks on this node that have not
* yet reached their replication factor. Otherwise returns false.
*/
public boolean isReplicationInProgress(DatanodeDescriptor srcNode) {
boolean isReplicationInProgress(DatanodeDescriptor srcNode) {
boolean status = false;
int underReplicatedBlocks = 0;
int decommissionOnlyReplicas = 0;
@ -1888,11 +1855,11 @@ public class BlockManager {
return status;
}
public int getActiveBlockCount() {
int getActiveBlockCount() {
return blocksMap.size() - (int)pendingDeletionBlocksCount;
}
public DatanodeDescriptor[] getNodes(BlockInfo block) {
DatanodeDescriptor[] getNodes(BlockInfo block) {
DatanodeDescriptor[] nodes =
new DatanodeDescriptor[block.numNodes()];
Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
@ -1902,22 +1869,22 @@ public class BlockManager {
return nodes;
}
public int getTotalBlocks() {
int getTotalBlocks() {
return blocksMap.size();
}
public void removeBlock(Block block) {
void removeBlock(Block block) {
addToInvalidates(block);
corruptReplicas.removeFromCorruptReplicasMap(block);
blocksMap.removeBlock(block);
}
public BlockInfo getStoredBlock(Block block) {
BlockInfo getStoredBlock(Block block) {
return blocksMap.getStoredBlock(block);
}
/* updates a block in under replication queue */
public void updateNeededReplications(Block block, int curReplicasDelta,
void updateNeededReplications(Block block, int curReplicasDelta,
int expectedReplicasDelta) {
namesystem.writeLock();
try {
@ -1938,13 +1905,13 @@ public class BlockManager {
}
}
public void checkReplication(Block block, int numExpectedReplicas) {
void checkReplication(Block block, int numExpectedReplicas) {
// filter out containingNodes that are marked for decommission.
NumberReplicas number = countNodes(block);
if (isNeededReplication(block, numExpectedReplicas, number.liveReplicas())) {
neededReplications.add(block,
number.liveReplicas(),
number.decommissionedReplicas(),
number.decommissionedReplicas,
numExpectedReplicas);
}
}
@ -1959,8 +1926,11 @@ public class BlockManager {
return fileINode.getReplication();
}
/** Remove a datanode from the invalidatesSet */
public void removeFromInvalidates(String storageID) {
/**
* Remove a datanode from the invalidatesSet
* @param n datanode
*/
void removeFromInvalidates(String storageID) {
Collection<Block> blocks = recentInvalidateSets.remove(storageID);
if (blocks != null) {
pendingDeletionBlocksCount -= blocks.size();
@ -2028,7 +1998,7 @@ public class BlockManager {
//Returns the number of racks over which a given block is replicated
//decommissioning/decommissioned nodes are not counted. corrupt replicas
//are also ignored
public int getNumberOfRacks(Block b) {
int getNumberOfRacks(Block b) {
HashSet<String> rackSet = new HashSet<String>(0);
Collection<DatanodeDescriptor> corruptNodes =
corruptReplicas.getNodes(b);
@ -2086,32 +2056,32 @@ public class BlockManager {
}
}
public long getMissingBlocksCount() {
long getMissingBlocksCount() {
// not locking
return this.neededReplications.getCorruptBlockSize();
}
public BlockInfo addINode(BlockInfo block, INodeFile iNode) {
BlockInfo addINode(BlockInfo block, INodeFile iNode) {
return blocksMap.addINode(block, iNode);
}
public INodeFile getINode(Block b) {
INodeFile getINode(Block b) {
return blocksMap.getINode(b);
}
public void removeFromCorruptReplicasMap(Block block) {
void removeFromCorruptReplicasMap(Block block) {
corruptReplicas.removeFromCorruptReplicasMap(block);
}
public int numCorruptReplicas(Block block) {
int numCorruptReplicas(Block block) {
return corruptReplicas.numCorruptReplicas(block);
}
public void removeBlockFromMap(Block block) {
void removeBlockFromMap(Block block) {
blocksMap.removeBlock(block);
}
public int getCapacity() {
int getCapacity() {
namesystem.readLock();
try {
return blocksMap.getCapacity();
@ -2134,7 +2104,7 @@ public class BlockManager {
* @return Up to numExpectedBlocks blocks from startingBlockId if it exists
*
*/
public long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
Long startingBlockId) {
return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks,
startingBlockId);
@ -2143,7 +2113,7 @@ public class BlockManager {
/**
* Return an iterator over the set of blocks for which there are no replicas.
*/
public BlockIterator getCorruptReplicaBlockIterator() {
UnderReplicatedBlocks.BlockIterator getCorruptReplicaBlockIterator() {
return neededReplications
.iterator(UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
}

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.util.ArrayList;
import java.util.Collection;
@ -26,8 +26,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.util.ReflectionUtils;
@ -96,7 +94,7 @@ public abstract class BlockPlacementPolicy {
* @return array of DatanodeDescriptor instances chosen as target
* and sorted as a pipeline.
*/
public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
abstract DatanodeDescriptor[] chooseTarget(String srcPath,
int numOfReplicas,
DatanodeDescriptor writer,
List<DatanodeDescriptor> chosenNodes,
@ -224,11 +222,11 @@ public abstract class BlockPlacementPolicy {
* @param numOfReplicas number of replicas wanted.
* @param writer the writer's machine, null if not in the cluster.
* @param blocksize size of the data to be written.
* @param excludedNodes datanodes that should not be considered as targets.
* @param excludedNodes: datanodes that should not be considered as targets.
* @return array of DatanodeDescriptor instances chosen as targets
* and sorted as a pipeline.
*/
public DatanodeDescriptor[] chooseTarget(String srcPath,
DatanodeDescriptor[] chooseTarget(String srcPath,
int numOfReplicas,
DatanodeDescriptor writer,
HashMap<Node, Node> excludedNodes,

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.util.ArrayList;
import java.util.Collection;
@ -32,9 +32,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
import org.apache.hadoop.hdfs.server.namenode.FSInodeInfo;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.net.NodeBase;
@ -92,7 +89,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
/** {@inheritDoc} */
@Override
public DatanodeDescriptor[] chooseTarget(String srcPath,
DatanodeDescriptor[] chooseTarget(String srcPath,
int numOfReplicas,
DatanodeDescriptor writer,
List<DatanodeDescriptor> chosenNodes,

View File

@ -15,12 +15,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.util.Iterator;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.namenode.INodeFile;
import org.apache.hadoop.hdfs.util.GSet;
import org.apache.hadoop.hdfs.util.LightWeightGSet;
@ -29,7 +28,7 @@ import org.apache.hadoop.hdfs.util.LightWeightGSet;
* block's metadata currently includes INode it belongs to and
* the datanodes that store the block.
*/
public class BlocksMap {
class BlocksMap {
private static class NodeIterator implements Iterator<DatanodeDescriptor> {
private BlockInfo blockInfo;
private int nextIdx = 0;
@ -101,7 +100,7 @@ public class BlocksMap {
/**
* Add block b belonging to the specified file inode to the map.
*/
public BlockInfo addINode(BlockInfo b, INodeFile iNode) {
BlockInfo addINode(BlockInfo b, INodeFile iNode) {
BlockInfo info = blocks.get(b);
if (info != b) {
info = b;
@ -137,7 +136,7 @@ public class BlocksMap {
* Searches for the block in the BlocksMap and
* returns Iterator that iterates through the nodes the block belongs to.
*/
public Iterator<DatanodeDescriptor> nodeIterator(Block b) {
Iterator<DatanodeDescriptor> nodeIterator(Block b) {
return nodeIterator(blocks.get(b));
}
@ -186,7 +185,7 @@ public class BlocksMap {
/**
* Check if the block exists in map
*/
public boolean contains(Block block) {
boolean contains(Block block) {
return blocks.contains(block);
}

View File

@ -15,11 +15,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.ipc.Server;
import java.util.*;

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.io.DataInput;
import java.io.IOException;
@ -26,7 +26,6 @@ import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.hdfs.DeprecatedUTF8;
import org.apache.hadoop.io.WritableUtils;
@ -45,7 +44,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
// Stores status of decommissioning.
// If node is not decommissioning, do not use this object for anything.
public DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
/** Block and targets pair */
@InterfaceAudience.Private
@ -97,8 +96,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
private int numBlocks = 0;
// isAlive == heartbeats.contains(this)
// This is an optimization, because contains takes O(n) time on Arraylist
public boolean isAlive = false;
public boolean needKeyUpdate = false;
protected boolean isAlive = false;
protected boolean needKeyUpdate = false;
/** A queue of blocks to be replicated by this datanode */
private BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
@ -205,7 +204,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
* Add datanode to the block.
* Add block to the head of the list of blocks belonging to the data-node.
*/
public boolean addBlock(BlockInfo b) {
boolean addBlock(BlockInfo b) {
if(!b.addNode(this))
return false;
// add to the head of the data-node list
@ -218,7 +217,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
* Remove block from the list of blocks belonging to the data-node.
* Remove datanode from the block.
*/
public boolean removeBlock(BlockInfo b) {
boolean removeBlock(BlockInfo b) {
blockList = b.listRemove(blockList, this);
if ( b.removeNode(this) ) {
numBlocks--;
@ -243,7 +242,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
* @param newBlock - a replacement block
* @return the new block
*/
public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
boolean done = removeBlock(oldBlock);
assert done : "Old block should belong to the data-node when replacing";
done = addBlock(newBlock);
@ -251,7 +250,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
return newBlock;
}
public void resetBlocks() {
void resetBlocks() {
this.capacity = 0;
this.remaining = 0;
this.blockPoolUsed = 0;
@ -269,7 +268,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
/**
* Updates stats from datanode heartbeat.
*/
public void updateHeartbeat(long capacity, long dfsUsed, long remaining,
void updateHeartbeat(long capacity, long dfsUsed, long remaining,
long blockPoolUsed, int xceiverCount, int volFailures) {
this.capacity = capacity;
this.dfsUsed = dfsUsed;
@ -284,7 +283,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
/**
* Iterates over the list of blocks belonging to the datanode.
*/
public static class BlockIterator implements Iterator<BlockInfo> {
static class BlockIterator implements Iterator<BlockInfo> {
private BlockInfo current;
private DatanodeDescriptor node;
@ -308,7 +307,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
}
}
public Iterator<BlockInfo> getBlockIterator() {
Iterator<BlockInfo> getBlockIterator() {
return new BlockIterator(this.blockList, this);
}
@ -362,11 +361,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
}
}
public List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
List<BlockTargetPair> getReplicationCommand(int maxTransfers) {
return replicateBlocks.poll(maxTransfers);
}
public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
if(blocks == null)
return null;
@ -376,7 +375,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
/**
* Remove the specified number of blocks to be invalidated
*/
public Block[] getInvalidateBlocks(int maxblocks) {
Block[] getInvalidateBlocks(int maxblocks) {
return getBlockArray(invalidateBlocks, maxblocks);
}
@ -419,7 +418,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
}
/** Serialization for FSEditLog */
public void readFieldsFromFSEditLog(DataInput in) throws IOException {
void readFieldsFromFSEditLog(DataInput in) throws IOException {
this.name = DeprecatedUTF8.readString(in);
this.storageID = DeprecatedUTF8.readString(in);
this.infoPort = in.readShort() & 0x0000ffff;
@ -446,7 +445,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
/**
* Increments counter for number of blocks scheduled.
*/
public void incBlocksScheduled() {
void incBlocksScheduled() {
currApproxBlocksScheduled++;
}
@ -486,13 +485,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
// by DatanodeID
return (this == obj) || super.equals(obj);
}
/** Decommissioning status */
public class DecommissioningStatus {
private int underReplicatedBlocks;
private int decommissionOnlyReplicas;
private int underReplicatedInOpenFiles;
private long startTime;
class DecommissioningStatus {
int underReplicatedBlocks;
int decommissionOnlyReplicas;
int underReplicatedInOpenFiles;
long startTime;
synchronized void set(int underRep,
int onlyRep, int underConstruction) {
@ -503,34 +501,32 @@ public class DatanodeDescriptor extends DatanodeInfo {
decommissionOnlyReplicas = onlyRep;
underReplicatedInOpenFiles = underConstruction;
}
/** @return the number of under-replicated blocks */
public synchronized int getUnderReplicatedBlocks() {
synchronized int getUnderReplicatedBlocks() {
if (isDecommissionInProgress() == false) {
return 0;
}
return underReplicatedBlocks;
}
/** @return the number of decommission-only replicas */
public synchronized int getDecommissionOnlyReplicas() {
synchronized int getDecommissionOnlyReplicas() {
if (isDecommissionInProgress() == false) {
return 0;
}
return decommissionOnlyReplicas;
}
/** @return the number of under-replicated blocks in open files */
public synchronized int getUnderReplicatedInOpenFiles() {
synchronized int getUnderReplicatedInOpenFiles() {
if (isDecommissionInProgress() == false) {
return 0;
}
return underReplicatedInOpenFiles;
}
/** Set start time */
public synchronized void setStartTime(long time) {
synchronized void setStartTime(long time) {
startTime = time;
}
/** @return start time */
public synchronized long getStartTime() {
synchronized long getStartTime() {
if (isDecommissionInProgress() == false) {
return 0;
}
@ -542,11 +538,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
* Set the flag to indicate if this datanode is disallowed from communicating
* with the namenode.
*/
public void setDisallowed(boolean flag) {
void setDisallowed(boolean flag) {
disallowed = flag;
}
/** Is the datanode disallowed from communicating with the namenode? */
public boolean isDisallowed() {
boolean isDisallowed() {
return disallowed;
}

View File

@ -21,7 +21,6 @@ import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.util.CyclicIteration;
/**

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
@ -26,40 +24,35 @@ import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnresolvedLinkException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.FSLimitException;
import org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
import org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
import org.apache.hadoop.hdfs.protocol.FSLimitException.*;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.util.ByteArray;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import org.apache.hadoop.hdfs.DFSConfigKeys;
/*************************************************
* FSDirectory stores the filesystem directory state.
@ -70,7 +63,7 @@ import org.apache.hadoop.hdfs.util.ByteArray;
* and logged to disk.
*
*************************************************/
public class FSDirectory implements Closeable {
class FSDirectory implements Closeable {
INodeDirectoryWithQuota rootDir;
FSImage fsImage;
@ -1339,7 +1332,7 @@ public class FSDirectory implements Closeable {
* @throws QuotaExceededException if the new count violates any quota limit
* @throws FileNotFound if path does not exist.
*/
public void updateSpaceConsumed(String path, long nsDelta, long dsDelta)
void updateSpaceConsumed(String path, long nsDelta, long dsDelta)
throws QuotaExceededException,
FileNotFoundException,
UnresolvedLinkException {

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import java.io.BufferedInputStream;
import java.io.DataInputStream;
import java.io.EOFException;
@ -34,30 +32,10 @@ import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.AddCloseOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.CancelDelegationTokenOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ClearNSQuotaOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ConcatDeleteOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.DeleteOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.GetDelegationTokenOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.MkdirOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.ReassignLeaseOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOldOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenameOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.RenewDelegationTokenOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetGenstampOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetNSQuotaOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetOwnerOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetPermissionsOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetQuotaOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SetReplicationOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.SymlinkOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TimesOp;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.UpdateMasterKeyOp;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.*;
public class FSEditLogLoader {
private final FSNamesystem fsNamesys;

View File

@ -32,8 +32,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.*;
import org.apache.hadoop.security.token.delegation.DelegationKey;
import org.apache.hadoop.io.BytesWritable;

View File

@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.protocol.LayoutVersion;
import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.io.MD5Hash;
import org.apache.hadoop.io.Text;

View File

@ -22,6 +22,7 @@ import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -32,9 +33,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DeprecatedUTF8;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;

View File

@ -97,20 +97,15 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor.BlockTargetPair;
import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
import org.apache.hadoop.hdfs.server.namenode.UnderReplicatedBlocks.BlockIterator;
import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
@ -240,7 +235,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
// Stores the correct file name hierarchy
//
public FSDirectory dir;
public BlockManager blockManager;
BlockManager blockManager;
// Block pool ID used by this namenode
String blockPoolId;
@ -275,10 +270,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
* Stores a set of DatanodeDescriptor objects.
* This is a subset of {@link #datanodeMap}, containing nodes that are
* considered alive.
* The HeartbeatMonitor periodically checks for out-dated entries,
* The {@link HeartbeatMonitor} periodically checks for outdated entries,
* and removes them from the list.
*/
public ArrayList<DatanodeDescriptor> heartbeats = new ArrayList<DatanodeDescriptor>();
ArrayList<DatanodeDescriptor> heartbeats = new ArrayList<DatanodeDescriptor>();
public LeaseManager leaseManager = new LeaseManager(this);
@ -319,8 +314,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
private volatile SafeModeInfo safeMode; // safe mode information
private Host2NodesMap host2DataNodeMap = new Host2NodesMap();
/** datanode network toplogy */
public NetworkTopology clusterMap = new NetworkTopology();
// datanode networktoplogy
NetworkTopology clusterMap = new NetworkTopology();
private DNSToSwitchMapping dnsToSwitchMapping;
private HostsFileReader hostsReader;
@ -334,7 +329,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
private final GenerationStamp generationStamp = new GenerationStamp();
// Ask Datanode only up to this many blocks to delete.
public int blockInvalidateLimit = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT;
int blockInvalidateLimit = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT;
// precision of access times.
private long accessTimePrecision = 0;
@ -477,23 +472,23 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
}
// utility methods to acquire and release read lock and write lock
public void readLock() {
void readLock() {
this.fsLock.readLock().lock();
}
public void readUnlock() {
void readUnlock() {
this.fsLock.readLock().unlock();
}
public void writeLock() {
void writeLock() {
this.fsLock.writeLock().lock();
}
public void writeUnlock() {
void writeUnlock() {
this.fsLock.writeLock().unlock();
}
public boolean hasWriteLock() {
boolean hasWriteLock() {
return this.fsLock.isWriteLockedByCurrentThread();
}
@ -1019,7 +1014,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
}
/** Create a LocatedBlock. */
public LocatedBlock createLocatedBlock(final Block b, final DatanodeInfo[] locations,
LocatedBlock createLocatedBlock(final Block b, final DatanodeInfo[] locations,
final long offset, final boolean corrupt) throws IOException {
return new LocatedBlock(getExtendedBlock(b), locations, offset, corrupt);
}
@ -3018,7 +3013,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
* @return an array of datanode commands
* @throws IOException
*/
public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
int xceiverCount, int xmitsInProgress, int failedVolumes)
throws IOException {
@ -3526,7 +3521,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
* If no such a node is available,
* then pick a node with least free space
*/
public void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess,
void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess,
Block b, short replication,
DatanodeDescriptor addedNode,
DatanodeDescriptor delNodeHint,
@ -3984,6 +3979,45 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
return replication;
}
/**
* A immutable object that stores the number of live replicas and
* the number of decommissined Replicas.
*/
static class NumberReplicas {
private int liveReplicas;
int decommissionedReplicas;
private int corruptReplicas;
private int excessReplicas;
NumberReplicas() {
initialize(0, 0, 0, 0);
}
NumberReplicas(int live, int decommissioned, int corrupt, int excess) {
initialize(live, decommissioned, corrupt, excess);
}
void initialize(int live, int decommissioned, int corrupt, int excess) {
liveReplicas = live;
decommissionedReplicas = decommissioned;
corruptReplicas = corrupt;
excessReplicas = excess;
}
int liveReplicas() {
return liveReplicas;
}
int decommissionedReplicas() {
return decommissionedReplicas;
}
int corruptReplicas() {
return corruptReplicas;
}
int excessReplicas() {
return excessReplicas;
}
}
/**
* Change, if appropriate, the admin state of a datanode to
* decommission completed. Return true if decommission is complete.
@ -4641,7 +4675,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
* Check whether the name node is in safe mode.
* @return true if safe mode is ON, false otherwise
*/
public boolean isInSafeMode() {
boolean isInSafeMode() {
// safeMode is volatile, and may be set to null at any time
SafeModeInfo safeMode = this.safeMode;
if (safeMode == null)
@ -4652,7 +4686,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
/**
* Check whether the name node is in startup mode.
*/
public boolean isInStartupSafeMode() {
boolean isInStartupSafeMode() {
// safeMode is volatile, and may be set to null at any time
SafeModeInfo safeMode = this.safeMode;
if (safeMode == null)
@ -4663,7 +4697,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
/**
* Check whether replication queues are populated.
*/
public boolean isPopulatingReplQueues() {
boolean isPopulatingReplQueues() {
// safeMode is volatile, and may be set to null at any time
SafeModeInfo safeMode = this.safeMode;
if (safeMode == null)
@ -4675,7 +4709,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
* Increment number of blocks that reached minimal replication.
* @param replication current replication
*/
public void incrementSafeBlockCount(int replication) {
void incrementSafeBlockCount(int replication) {
// safeMode is volatile, and may be set to null at any time
SafeModeInfo safeMode = this.safeMode;
if (safeMode == null)
@ -4686,7 +4720,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
/**
* Decrement number of blocks that reached minimal replication.
*/
public void decrementSafeBlockCount(Block b) {
void decrementSafeBlockCount(Block b) {
// safeMode is volatile, and may be set to null at any time
SafeModeInfo safeMode = this.safeMode;
if (safeMode == null) // mostly true
@ -5008,13 +5042,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
@Override // FSNamesystemMBean
@Metric
public long getPendingReplicationBlocks() {
return blockManager.getPendingReplicationBlocksCount();
return blockManager.pendingReplicationBlocksCount;
}
@Override // FSNamesystemMBean
@Metric
public long getUnderReplicatedBlocks() {
return blockManager.getUnderReplicatedBlocksCount();
return blockManager.underReplicatedBlocksCount;
}
/** Return number of under-replicated but not missing blocks */
@ -5025,23 +5059,23 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
/** Returns number of blocks with corrupt replicas */
@Metric({"CorruptBlocks", "Number of blocks with corrupt replicas"})
public long getCorruptReplicaBlocks() {
return blockManager.getCorruptReplicaBlocksCount();
return blockManager.corruptReplicaBlocksCount;
}
@Override // FSNamesystemMBean
@Metric
public long getScheduledReplicationBlocks() {
return blockManager.getScheduledReplicationBlocksCount();
return blockManager.scheduledReplicationBlocksCount;
}
@Metric
public long getPendingDeletionBlocks() {
return blockManager.getPendingDeletionBlocksCount();
return blockManager.pendingDeletionBlocksCount;
}
@Metric
public long getExcessBlocks() {
return blockManager.getExcessBlocksCount();
return blockManager.excessBlocksCount;
}
@Metric
@ -5410,7 +5444,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
}
/** Get a datanode descriptor given corresponding storageID */
public DatanodeDescriptor getDatanode(String nodeID) {
DatanodeDescriptor getDatanode(String nodeID) {
assert hasReadOrWriteLock();
return datanodeMap.get(nodeID);
}
@ -5474,7 +5508,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
if (startBlockAfter != null) {
startBlockId = Block.filename2id(startBlockAfter);
}
UnderReplicatedBlocks.BlockIterator blkIterator = blockManager.getCorruptReplicaBlockIterator();
BlockIterator blkIterator = blockManager.getCorruptReplicaBlockIterator();
while (blkIterator.hasNext()) {
Block blk = blkIterator.next();
INode inode = blockManager.getINode(blk);

View File

@ -17,13 +17,10 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import java.util.HashMap;
import java.util.Random;
import java.util.*;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
class Host2NodesMap {
private HashMap<String, DatanodeDescriptor[]> map
= new HashMap<String, DatanodeDescriptor[]>();

View File

@ -20,13 +20,11 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.permission.*;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.util.StringUtils;
/**
@ -34,7 +32,7 @@ import org.apache.hadoop.util.StringUtils;
* This is a base INode class containing common fields for file and
* directory inodes.
*/
public abstract class INode implements Comparable<byte[]>, FSInodeInfo {
abstract class INode implements Comparable<byte[]>, FSInodeInfo {
/*
* The inode name is in java UTF8 encoding;
* The name in HdfsFileStatus should keep the same encoding as this.
@ -326,7 +324,7 @@ public abstract class INode implements Comparable<byte[]>, FSInodeInfo {
/**
* Is this inode being constructed?
*/
public boolean isUnderConstruction() {
boolean isUnderConstruction() {
return false;
}

View File

@ -24,11 +24,8 @@ import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
/** I-node for closed file. */
public class INodeFile extends INode {
class INodeFile extends INode {
static final FsPermission UMASK = FsPermission.createImmutable((short)0111);
//Number of bits for Block size
@ -109,7 +106,7 @@ public class INodeFile extends INode {
* Get file blocks
* @return file blocks
*/
public BlockInfo[] getBlocks() {
BlockInfo[] getBlocks() {
return this.blocks;
}
@ -152,7 +149,7 @@ public class INodeFile extends INode {
/**
* Set file block
*/
public void setBlock(int idx, BlockInfo blk) {
void setBlock(int idx, BlockInfo blk) {
this.blocks[idx] = blk;
}
@ -240,7 +237,7 @@ public class INodeFile extends INode {
* Get the last block of the file.
* Make sure it has the right type.
*/
public <T extends BlockInfo> T getLastBlock() throws IOException {
<T extends BlockInfo> T getLastBlock() throws IOException {
if (blocks == null || blocks.length == 0)
return null;
T returnBlock = null;
@ -255,8 +252,7 @@ public class INodeFile extends INode {
return returnBlock;
}
/** @return the number of blocks */
public int numBlocks() {
int numBlocks() {
return blocks == null ? 0 : blocks.length;
}
}

View File

@ -21,15 +21,10 @@ import java.io.IOException;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
/**
* I-node for file being written.
*/
public class INodeFileUnderConstruction extends INodeFile {
class INodeFileUnderConstruction extends INodeFile {
private String clientName; // lease holder
private final String clientMachine;
private final DatanodeDescriptor clientNode; // if client is a cluster node too.
@ -48,7 +43,7 @@ public class INodeFileUnderConstruction extends INodeFile {
this.clientNode = clientNode;
}
INodeFileUnderConstruction(byte[] name,
public INodeFileUnderConstruction(byte[] name,
short blockReplication,
long modificationTime,
long preferredBlockSize,
@ -85,7 +80,7 @@ public class INodeFileUnderConstruction extends INodeFile {
* Is this inode being constructed?
*/
@Override
public boolean isUnderConstruction() {
boolean isUnderConstruction() {
return true;
}
@ -127,7 +122,7 @@ public class INodeFileUnderConstruction extends INodeFile {
* Convert the last block of the file to an under-construction block.
* Set its locations.
*/
public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeDescriptor[] targets)
throws IOException {
if (blocks == null || blocks.length == 0) {

View File

@ -163,7 +163,7 @@ public class LeaseManager {
/**
* Finds the pathname for the specified pendingFile
*/
public synchronized String findPath(INodeFileUnderConstruction pendingFile)
synchronized String findPath(INodeFileUnderConstruction pendingFile)
throws IOException {
Lease lease = getLease(pendingFile.getClientName());
if (lease != null) {

View File

@ -247,7 +247,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
/** Return the {@link FSNamesystem} object.
* @return {@link FSNamesystem} object.
*/
public FSNamesystem getNamesystem() {
FSNamesystem getNamesystem() {
return namesystem;
}

View File

@ -36,20 +36,20 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.BlockReader;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
/**

View File

@ -36,24 +36,24 @@ import javax.servlet.http.HttpServletResponse;
import javax.servlet.jsp.JspWriter;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.FSConstants.UpgradeAction;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.ServletUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.VersionInfo;
import org.znerd.xmlenc.XMLOutputter;
import org.znerd.xmlenc.*;
class NamenodeJspHelper {
static String getSafeModeText(FSNamesystem fsn) {

View File

@ -15,11 +15,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import static org.apache.hadoop.hdfs.server.common.Util.now;
import org.apache.hadoop.util.*;
import java.io.*;

View File

@ -15,18 +15,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.util.*;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
/* Class for keeping track of under replication blocks
* Blocks have replication priority, with priority 0 indicating the highest
* Blocks have only one replicas has the highest
*/
public class UnderReplicatedBlocks implements Iterable<Block> {
class UnderReplicatedBlocks implements Iterable<Block> {
static final int LEVEL = 5;
static public final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
private List<TreeSet<Block>> priorityQueues = new ArrayList<TreeSet<Block>>();
@ -48,7 +47,7 @@ public class UnderReplicatedBlocks implements Iterable<Block> {
}
/* Return the total number of under replication blocks */
public synchronized int size() {
synchronized int size() {
int size = 0;
for (int i=0; i<LEVEL; i++) {
size += priorityQueues.get(i).size();
@ -71,7 +70,7 @@ public class UnderReplicatedBlocks implements Iterable<Block> {
}
/* Check if a block is in the neededReplication queue */
public synchronized boolean contains(Block block) {
synchronized boolean contains(Block block) {
for(TreeSet<Block> set:priorityQueues) {
if(set.contains(block)) { return true; }
}
@ -219,7 +218,7 @@ public class UnderReplicatedBlocks implements Iterable<Block> {
return new BlockIterator();
}
public class BlockIterator implements Iterator<Block> {
class BlockIterator implements Iterator<Block> {
private int level;
private boolean isIteratorForLevel = false;
private List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();

View File

@ -17,20 +17,15 @@
*/
package org.apache.hadoop.hdfs.server.protocol;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.io.*;
import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableFactories;
import org.apache.hadoop.io.WritableFactory;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor.BlockTargetPair;
import org.apache.hadoop.io.*;
/****************************************************

View File

@ -61,10 +61,10 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;

View File

@ -25,7 +25,7 @@ import junit.framework.TestCase;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
/**
* This class tests DatanodeDescriptor.getBlocksScheduled() at the

View File

@ -17,31 +17,30 @@
*/
package org.apache.hadoop.hdfs.server.datanode;
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assume.assumeTrue;
import java.io.File;
import java.util.ArrayList;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import static org.apache.hadoop.test.MetricsAsserts.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.log4j.Level;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.*;
import static org.junit.Assume.assumeTrue;
/**
* Test reporting of DN volume failure counts and metrics.

View File

@ -23,7 +23,6 @@ import java.io.IOException;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.Storage;

View File

@ -19,11 +19,10 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.ipc.Server;
/**
* This is a utility class to expose NameNode functionality for unit tests.

View File

@ -34,8 +34,6 @@ import org.apache.hadoop.hdfs.TestFileCreation;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
import org.junit.AfterClass;
import org.junit.BeforeClass;

View File

@ -17,15 +17,14 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import junit.framework.TestCase;
/**
* Test if FSNamesystem handles heartbeat right
*/

View File

@ -15,20 +15,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.*;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.namenode.CorruptReplicasMap;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
/**
@ -82,6 +79,7 @@ public class TestCorruptReplicaInfo extends TestCase {
DatanodeDescriptor dn1 = new DatanodeDescriptor();
DatanodeDescriptor dn2 = new DatanodeDescriptor();
DatanodeDescriptor dn3 = new DatanodeDescriptor();
crm.addToCorruptReplicasMap(getBlock(0), dn1);
assertEquals("Number of corrupt blocks not returning correctly",

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.util.ArrayList;

View File

@ -29,9 +29,9 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;

View File

@ -17,15 +17,19 @@
*/
package org.apache.hadoop.hdfs.server.namenode;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.Random;
import org.junit.BeforeClass;
import org.junit.AfterClass;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
@ -36,10 +40,8 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.FSConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
/**
* This class tests the decommissioning of nodes.

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.util.ArrayList;
@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;

View File

@ -18,10 +18,9 @@
package org.apache.hadoop.hdfs.server.namenode;
import junit.framework.TestCase;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import junit.framework.TestCase;
public class TestHost2NodesMap extends TestCase {
static private Host2NodesMap map = new Host2NodesMap();

View File

@ -21,17 +21,17 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.File;
import java.util.ArrayList;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DF;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import junit.framework.TestCase;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;

View File

@ -21,8 +21,6 @@ import java.util.Collection;
import java.util.Iterator;
import java.util.concurrent.TimeoutException;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -32,8 +30,9 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
import junit.framework.TestCase;
/**
* Test if live nodes count per node is correct

View File

@ -20,21 +20,20 @@ package org.apache.hadoop.hdfs.server.namenode;
import java.io.File;
import java.io.IOException;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
import org.apache.hadoop.hdfs.TestDatanodeBlockScanner;
import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import junit.framework.TestCase;
public class TestOverReplicatedBlocks extends TestCase {
/** Test processOverReplicatedBlock can handle corrupt replicas fine.

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import junit.framework.TestCase;
import java.lang.System;

View File

@ -15,7 +15,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
package org.apache.hadoop.hdfs.server.namenode;
import java.io.IOException;
import java.util.ArrayList;
@ -23,19 +24,18 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import junit.framework.TestCase;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.FSConstants;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node;
import junit.framework.TestCase;
public class TestReplicationPolicy extends TestCase {
private static final int BLOCK_SIZE = 1024;

View File

@ -15,9 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
import junit.framework.TestCase;
package org.apache.hadoop.hdfs.server.namenode;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -27,7 +25,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import junit.framework.TestCase;
public class TestUnderReplicatedBlocks extends TestCase {
public void testSetrepIncWithUnderReplicatedBlocks() throws Exception {

View File

@ -17,35 +17,34 @@
*/
package org.apache.hadoop.hdfs.server.namenode.metrics;
import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
import static org.apache.hadoop.test.MetricsAsserts.assertGauge;
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
import java.io.DataInputStream;
import java.io.IOException;
import java.util.Random;
import junit.framework.TestCase;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.BlockManager;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.test.MetricsAsserts;
import org.apache.log4j.Level;
import org.apache.commons.logging.LogFactory;
import static org.apache.hadoop.test.MetricsAsserts.*;
/**
* Test for metrics published by the Namenode
*/

View File

@ -24,8 +24,8 @@ import java.util.Map;
import junit.framework.TestCase;
import org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
public class TestNetworkTopology extends TestCase {
private final static NetworkTopology cluster = new NetworkTopology();

View File

@ -19,16 +19,6 @@
package org.apache.hadoop.hdfs.server.namenode;
import static junit.framework.Assert.assertTrue;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import java.io.File;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.impl.Log4JLogger;
@ -44,16 +34,20 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.common.HdfsConstants;
import org.apache.hadoop.hdfs.server.common.HdfsConstants.NamenodeRole;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.log4j.Level;
import org.junit.After;
import static org.junit.Assert.*;
import org.junit.Before;
import org.junit.Test;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.*;
import java.io.File;
import java.io.IOException;
public class TestNNLeaseRecovery {
private static final Log LOG = LogFactory.getLog(TestNNLeaseRecovery.class);

View File

@ -59,9 +59,19 @@
%>
<%@ page
contentType="application/xml"
import="java.io.IOException"
import="java.util.Iterator"
import="org.apache.hadoop.conf.Configuration"
import="org.apache.hadoop.hdfs.protocol.Block"
import="org.apache.hadoop.hdfs.server.namenode.INode"
import="org.apache.hadoop.hdfs.server.namenode.BlocksMap"
import="org.apache.hadoop.hdfs.server.namenode.BlockInfo"
import="org.apache.hadoop.hdfs.server.namenode.DatanodeDescriptor"
import="org.apache.hadoop.hdfs.server.namenode.NamenodeJspHelper.XMLBlockInfo"
import="org.apache.hadoop.hdfs.server.common.JspHelper"
import="org.apache.hadoop.util.ServletUtil"
import="org.znerd.xmlenc.*"
%>
<%!
//for java.io.Serializable