HDFS-2832. Merge r1550363 from trunk for merging HDFS Heterogeneous Storage feature phase 1.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1556076 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arpit Agarwal 2014-01-06 23:58:33 +00:00
parent 9bf09c63f9
commit 4f044fc5ff
119 changed files with 3912 additions and 2159 deletions

View File

@ -30,6 +30,10 @@ Release 2.4.0 - UNRELEASED
HDFS-5514. FSNamesystem's fsLock should allow custom implementation (daryn)
HDFS-2832. Heterogeneous Storages support in HDFS phase 1 - treat DataNode
as a collection of storages (see breakdown of tasks below for features and
contributors).
IMPROVEMENTS
HDFS-5267. Remove volatile from LightWeightHashSet. (Junping Du via llu)
@ -251,6 +255,131 @@ Release 2.4.0 - UNRELEASED
HDFS-5690. DataNode fails to start in secure mode when dfs.http.policy equals to
HTTP_ONLY. (Haohui Mai via jing9)
BREAKDOWN OF HDFS-2832 SUBTASKS AND RELATED JIRAS
HDFS-4985. Add storage type to the protocol and expose it in block report
and block locations. (Arpit Agarwal)
HDFS-5115. Make StorageID a UUID. (Arpit Agarwal)
HDFS-5000. DataNode configuration should allow specifying storage type.
(Arpit Agarwal)
HDFS-4987. Namenode changes to track multiple storages per datanode.
(szetszwo)
HDFS-5154. Fix TestBlockManager and TestDatanodeDescriptor after HDFS-4987.
(Junping Du via szetszwo)
HDFS-5009. Include storage information in the LocatedBlock. (szetszwo)
HDFS-5134. Move blockContentsStale, heartbeatedSinceFailover and
firstBlockReport from DatanodeDescriptor to DatanodeStorageInfo; and
fix a synchronization problem in DatanodeStorageInfo. (szetszwo)
HDFS-5157. Add StorageType to FsVolume. (Junping Du via szetszwo)
HDFS-4990. Change BlockPlacementPolicy to choose storages instead of
datanodes. (szetszwo)
HDFS-5232. Protocol changes to transmit StorageUuid. (Arpit Agarwal)
HDFS-5233. Use Datanode UUID to identify Datanodes. (Arpit Agarwal)
HDFS-5222. Move block schedule information from DatanodeDescriptor to
DatanodeStorageInfo. (szetszwo)
HDFS-4988. Datanode must support all the volumes as individual storages.
(Arpit Agarwal)
HDFS-5377. Heartbeats from Datandode should include one storage report
per storage directory. (Arpit Agarwal)
HDFS-5398. NameNode changes to process storage reports per storage
directory. (Arpit Agarwal)
HDFS-5390. Send one incremental block report per storage directory.
(Arpit Agarwal)
HDFS-5401. Fix NPE in Directory Scanner. (Arpit Agarwal)
HDFS-5417. Fix storage IDs in PBHelper and UpgradeUtilities. (szetszwo)
HDFS-5214. Fix NPEs in BlockManager and DirectoryScanner. (Arpit Agarwal)
HDFS-5435. File append fails to initialize storageIDs. (Junping Du via
Arpit Agarwal)
HDFS-5437. Fix TestBlockReport and TestBPOfferService failures. (Arpit
Agarwal)
HDFS-5447. Fix TestJspHelper. (Arpit Agarwal)
HDFS-5452. Fix TestReplicationPolicy and TestBlocksScheduledCounter.
HDFS-5448. Datanode should generate its ID on first registration. (Arpit
Agarwal)
HDFS-5448. Fix break caused by previous checkin for HDFS-5448. (Arpit
Agarwal)
HDFS-5455. NN should update storageMap on first heartbeat. (Arpit Agarwal)
HDFS-5457. Fix TestDatanodeRegistration, TestFsck and TestAddBlockRetry.
(Contributed by szetszwo)
HDFS-5466. Update storage IDs when the pipeline is updated. (Contributed
by szetszwo)
HDFS-5439. Fix TestPendingReplication. (Contributed by Junping Du, Arpit
Agarwal)
HDFS-5470. Add back trunk's reportDiff algorithm to the branch.
(Contributed by szetszwo)
HDFS-5472. Fix TestDatanodeManager, TestSafeMode and
TestNNThroughputBenchmark (Contributed by szetszwo)
HDFS-5475. NN incorrectly tracks more than one replica per DN. (Arpit
Agarwal)
HDFS-5481. Fix TestDataNodeVolumeFailure in branch HDFS-2832. (Contributed
by Junping Du)
HDFS-5480. Update Balancer for HDFS-2832. (Contributed by szetszwo)
HDFS-5486. Fix TestNameNodeMetrics for HDFS-2832. (Arpit Agarwal)
HDFS-5491. Update editsStored for HDFS-2832. (Arpit Agarwal)
HDFS-5494. Fix findbugs warnings for HDFS-2832. (Arpit Agarwal)
HDFS-5508. Fix compilation error after merge. (Contributed by szetszwo)
HDFS-5501. Fix pendingReceivedRequests tracking in BPServiceActor. (Arpit
Agarwal)
HDFS-5510. Fix a findbug warning in DataStorage.java on HDFS-2832 branch.
(Junping Du via Arpit Agarwal)
HDFS-5515. Fix TestDFSStartupVersions for HDFS-2832. (Arpit Agarwal)
HDFS-5527. Fix TestUnderReplicatedBlocks on branch HDFS-2832. (Arpit
Agarwal)
HDFS-5547. Fix build break after merge from trunk to HDFS-2832. (Arpit
Agarwal)
HDFS-5542. Fix TODO and clean up the code in HDFS-2832. (Contributed by
szetszwo)
HDFS-5559. Fix TestDatanodeConfig in HDFS-2832. (Contributed by szetszwo)
HDFS-5484. StorageType and State in DatanodeStorageInfo in NameNode is
not accurate. (Eric Sirianni via Arpit Agarwal)
HDFS-5648. Get rid of FsDatasetImpl#perVolumeReplicaMap. (Arpit Agarwal)
Release 2.3.0 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -1383,7 +1383,7 @@ public synchronized boolean seekToNewSource(long targetPos) throws IOException {
* deadNodes and added currentNode again. Thats ok. */
deadNodes.remove(oldNode);
}
if (!oldNode.getStorageID().equals(newNode.getStorageID())) {
if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
currentNode = newNode;
return true;
} else {

View File

@ -321,6 +321,7 @@ class DataStreamer extends Daemon {
private DataInputStream blockReplyStream;
private ResponseProcessor response = null;
private volatile DatanodeInfo[] nodes = null; // list of targets for current block
private volatile String[] storageIDs = null;
private LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes =
CacheBuilder.newBuilder()
.expireAfterWrite(
@ -411,7 +412,7 @@ private DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat,
}
// setup pipeline to append to the last block XXX retries??
nodes = lastBlock.getLocations();
setPipeline(lastBlock);
errorIndex = -1; // no errors yet.
if (nodes.length < 1) {
throw new IOException("Unable to retrieve blocks locations " +
@ -420,6 +421,14 @@ private DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat,
}
}
private void setPipeline(LocatedBlock lb) {
setPipeline(lb.getLocations(), lb.getStorageIDs());
}
private void setPipeline(DatanodeInfo[] nodes, String[] storageIDs) {
this.nodes = nodes;
this.storageIDs = storageIDs;
}
private void setFavoredNodes(String[] favoredNodes) {
this.favoredNodes = favoredNodes;
@ -443,7 +452,7 @@ private void endBlock() {
this.setName("DataStreamer for file " + src);
closeResponder();
closeStream();
nodes = null;
setPipeline(null, null);
stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
}
@ -510,7 +519,7 @@ public void run() {
if(DFSClient.LOG.isDebugEnabled()) {
DFSClient.LOG.debug("Allocating new block");
}
nodes = nextBlockOutputStream();
setPipeline(nextBlockOutputStream());
initDataStreaming();
} else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
if(DFSClient.LOG.isDebugEnabled()) {
@ -922,9 +931,10 @@ private void addDatanode2ExistingPipeline() throws IOException {
//get a new datanode
final DatanodeInfo[] original = nodes;
final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
src, block, nodes, failed.toArray(new DatanodeInfo[failed.size()]),
src, block, nodes, storageIDs,
failed.toArray(new DatanodeInfo[failed.size()]),
1, dfsClient.clientName);
nodes = lb.getLocations();
setPipeline(lb);
//find the new datanode
final int d = findNewDatanode(original);
@ -1024,7 +1034,14 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException {
System.arraycopy(nodes, 0, newnodes, 0, errorIndex);
System.arraycopy(nodes, errorIndex+1, newnodes, errorIndex,
newnodes.length-errorIndex);
nodes = newnodes;
final String[] newStorageIDs = new String[newnodes.length];
System.arraycopy(storageIDs, 0, newStorageIDs, 0, errorIndex);
System.arraycopy(storageIDs, errorIndex+1, newStorageIDs, errorIndex,
newStorageIDs.length-errorIndex);
setPipeline(newnodes, newStorageIDs);
hasError = false;
lastException.set(null);
errorIndex = -1;
@ -1060,7 +1077,8 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException {
// update pipeline at the namenode
ExtendedBlock newBlock = new ExtendedBlock(
block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock, nodes);
dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
nodes, storageIDs);
// update client side generation stamp
block = newBlock;
}
@ -1073,7 +1091,7 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException {
* Must get block ID and the IDs of the destinations from the namenode.
* Returns the list of target datanodes.
*/
private DatanodeInfo[] nextBlockOutputStream() throws IOException {
private LocatedBlock nextBlockOutputStream() throws IOException {
LocatedBlock lb = null;
DatanodeInfo[] nodes = null;
int count = dfsClient.getConf().nBlockWriteRetry;
@ -1115,7 +1133,7 @@ private DatanodeInfo[] nextBlockOutputStream() throws IOException {
if (!success) {
throw new IOException("Unable to create new block.");
}
return nodes;
return lb;
}
// connects to the first datanode in the pipeline

View File

@ -145,6 +145,23 @@ public static SecureRandom getSecureRandom() {
return SECURE_RANDOM.get();
}
/** Shuffle the elements in the given array. */
public static <T> T[] shuffle(final T[] array) {
if (array != null && array.length > 0) {
final Random random = getRandom();
for (int n = array.length; n > 1; ) {
final int randomIndex = random.nextInt(n);
n--;
if (n != randomIndex) {
final T tmp = array[randomIndex];
array[randomIndex] = array[n];
array[n] = tmp;
}
}
}
return array;
}
/**
* Compartor for sorting DataNodeInfo[] based on decommissioned states.
* Decommissioned nodes are moved to the end of the array on sorting with

View File

@ -0,0 +1,35 @@
/**
* 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;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
/**
* Defines the types of supported storage media. The default storage
* medium is assumed to be DISK.
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable
public enum StorageType {
DISK,
SSD;
public static StorageType DEFAULT = DISK;
}

View File

@ -19,7 +19,9 @@
import java.util.Iterator;
import java.util.List;
import java.util.Random;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@ -250,33 +252,28 @@ private ReplicaState blockReplicaState(int index) {
}
/**
* The block-id of the indexTh block
* @param index - the block whose block-id is desired
* @return the block-id
* Corrupt the generation stamp of the block with the given index.
* Not meant to be used outside of tests.
*/
@Deprecated
public long getBlockId(final int index) {
return blockId(index);
}
/**
* The block-len of the indexTh block
* @param index - the block whose block-len is desired
* @return - the block-len
*/
@Deprecated
public long getBlockLen(final int index) {
return blockLength(index);
@VisibleForTesting
public long corruptBlockGSForTesting(final int blockIndex, Random rand) {
long oldGS = blockList[index2BlockId(blockIndex) + 2];
while (blockList[index2BlockId(blockIndex) + 2] == oldGS) {
blockList[index2BlockId(blockIndex) + 2] = rand.nextInt();
}
return oldGS;
}
/**
* The generation stamp of the indexTh block
* @param index - the block whose block-len is desired
* @return - the generation stamp
* Corrupt the length of the block with the given index by truncation.
* Not meant to be used outside of tests.
*/
@Deprecated
public long getBlockGenStamp(final int index) {
return blockGenerationStamp(index);
@VisibleForTesting
public long corruptBlockLengthForTesting(final int blockIndex, Random rand) {
long oldLength = blockList[index2BlockId(blockIndex) + 1];
blockList[index2BlockId(blockIndex) + 1] =
rand.nextInt((int) oldLength - 1);
return oldLength;
}
/**

View File

@ -353,7 +353,8 @@ public LocatedBlock addBlock(String src, String clientName,
*/
@Idempotent
public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
final DatanodeInfo[] existings, final String[] existingStorageIDs,
final DatanodeInfo[] excludes,
final int numAdditionalNodes, final String clientName
) throws AccessControlException, FileNotFoundException,
SafeModeException, UnresolvedLinkException, IOException;
@ -982,7 +983,7 @@ public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
*/
@AtMostOnce
public void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes)
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
throws IOException;
/**

View File

@ -21,6 +21,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import com.google.common.annotations.VisibleForTesting;
/**
* This class represents the primary identifier for a Datanode.
* Datanodes are identified by how they can be contacted (hostname
@ -40,37 +42,46 @@ public class DatanodeID implements Comparable<DatanodeID> {
private String ipAddr; // IP address
private String hostName; // hostname claimed by datanode
private String peerHostName; // hostname from the actual connection
private String storageID; // unique per cluster storageID
private int xferPort; // data streaming port
private int infoPort; // info server port
private int infoSecurePort; // info server port
private int ipcPort; // IPC server port
/**
* UUID identifying a given datanode. For upgraded Datanodes this is the
* same as the StorageID that was previously used by this Datanode.
* For newly formatted Datanodes it is a UUID.
*/
private String datanodeUuid = null;
public DatanodeID(DatanodeID from) {
this(from.getIpAddr(),
from.getHostName(),
from.getStorageID(),
from.getDatanodeUuid(),
from.getXferPort(),
from.getInfoPort(),
from.getInfoSecurePort(),
from.getIpcPort());
this.peerHostName = from.getPeerHostName();
}
/**
* Create a DatanodeID
* @param ipAddr IP
* @param hostName hostname
* @param storageID data storage ID
* @param datanodeUuid data node ID, UUID for new Datanodes, may be the
* storage ID for pre-UUID datanodes. NULL if unknown
* e.g. if this is a new datanode. A new UUID will
* be assigned by the namenode.
* @param xferPort data transfer port
* @param infoPort info server port
* @param ipcPort ipc server port
*/
public DatanodeID(String ipAddr, String hostName, String storageID,
public DatanodeID(String ipAddr, String hostName, String datanodeUuid,
int xferPort, int infoPort, int infoSecurePort, int ipcPort) {
this.ipAddr = ipAddr;
this.hostName = hostName;
this.storageID = storageID;
this.datanodeUuid = checkDatanodeUuid(datanodeUuid);
this.xferPort = xferPort;
this.infoPort = infoPort;
this.infoSecurePort = infoSecurePort;
@ -85,8 +96,24 @@ public void setPeerHostName(String peerHostName) {
this.peerHostName = peerHostName;
}
public void setStorageID(String storageID) {
this.storageID = storageID;
/**
* @return data node ID.
*/
public String getDatanodeUuid() {
return datanodeUuid;
}
@VisibleForTesting
public void setDatanodeUuidForTesting(String datanodeUuid) {
this.datanodeUuid = datanodeUuid;
}
private String checkDatanodeUuid(String uuid) {
if (uuid == null || uuid.isEmpty()) {
return null;
} else {
return uuid;
}
}
/**
@ -168,13 +195,6 @@ public String getIpcAddr(boolean useHostname) {
return useHostname ? getIpcAddrWithHostname() : getIpcAddr();
}
/**
* @return data storage ID.
*/
public String getStorageID() {
return storageID;
}
/**
* @return xferPort (the port for data streaming)
*/
@ -212,12 +232,12 @@ public boolean equals(Object to) {
return false;
}
return (getXferAddr().equals(((DatanodeID)to).getXferAddr()) &&
storageID.equals(((DatanodeID)to).getStorageID()));
datanodeUuid.equals(((DatanodeID)to).getDatanodeUuid()));
}
@Override
public int hashCode() {
return getXferAddr().hashCode()^ storageID.hashCode();
return getXferAddr().hashCode()^ datanodeUuid.hashCode();
}
@Override

View File

@ -108,7 +108,7 @@ public DatanodeInfo(DatanodeID nodeID, String location,
final long capacity, final long dfsUsed, final long remaining,
final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
final AdminStates adminState) {
this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getStorageID(),
this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getDatanodeUuid(),
nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
lastUpdate, xceiverCount, location, adminState);
@ -116,12 +116,12 @@ public DatanodeInfo(DatanodeID nodeID, String location,
/** Constructor */
public DatanodeInfo(final String ipAddr, final String hostName,
final String storageID, final int xferPort, final int infoPort,
final String datanodeUuid, final int xferPort, final int infoPort,
final int infoSecurePort, final int ipcPort,
final long capacity, final long dfsUsed, final long remaining,
final long blockPoolUsed, final long lastUpdate, final int xceiverCount,
final String networkLocation, final AdminStates adminState) {
super(ipAddr, hostName, storageID, xferPort, infoPort,
super(ipAddr, hostName, datanodeUuid, xferPort, infoPort,
infoSecurePort, ipcPort);
this.capacity = capacity;
this.dfsUsed = dfsUsed;

View File

@ -106,8 +106,10 @@ public static enum Feature {
SEQUENTIAL_BLOCK_ID(-46, "Allocate block IDs sequentially and store " +
"block IDs in the edits log and image files"),
EDITLOG_SUPPORT_RETRYCACHE(-47, "Record ClientId and CallId in editlog to "
+ "enable rebuilding retry cache in case of HA failover");
+ "enable rebuilding retry cache in case of HA failover"),
ADD_DATANODE_AND_STORAGE_UUIDS(-49, -47, "Replace StorageID with DatanodeUuid."
+ " Use distinct StorageUuid per storage directory.", false);
final int lv;
final int ancestorLV;
final String description;
@ -247,3 +249,4 @@ public static int getCurrentLayoutVersion() {
throw new AssertionError("All layout versions are reserved.");
}
}

View File

@ -19,7 +19,9 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.security.token.Token;
/**
@ -34,6 +36,10 @@ public class LocatedBlock {
private ExtendedBlock b;
private long offset; // offset of the first byte of the block in the file
private DatanodeInfo[] locs;
/** Storage ID for each replica */
private String[] storageIDs;
// Storage type for each replica, if reported.
private StorageType[] storageTypes;
// corrupt flag is true if all of the replicas of a block are corrupt.
// else false. If block has few corrupt replicas, they are filtered and
// their locations are not part of this object
@ -44,11 +50,30 @@ public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
this(b, locs, -1, false); // startOffset is unknown
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset) {
this(b, locs, startOffset, false);
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
boolean corrupt) {
this(b, locs, null, null, startOffset, corrupt);
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages) {
this(b, storages, -1, false); // startOffset is unknown
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
String[] storageIDs, StorageType[] storageTypes) {
this(b, locs, storageIDs, storageTypes, -1, false);
}
public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
long startOffset, boolean corrupt) {
this(b, DatanodeStorageInfo.toDatanodeInfos(storages),
DatanodeStorageInfo.toStorageIDs(storages),
DatanodeStorageInfo.toStorageTypes(storages),
startOffset, corrupt); // startOffset is unknown
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
StorageType[] storageTypes, long startOffset,
boolean corrupt) {
this.b = b;
this.offset = startOffset;
@ -58,6 +83,8 @@ public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
} else {
this.locs = locs;
}
this.storageIDs = storageIDs;
this.storageTypes = storageTypes;
}
public Token<BlockTokenIdentifier> getBlockToken() {
@ -75,7 +102,15 @@ public ExtendedBlock getBlock() {
public DatanodeInfo[] getLocations() {
return locs;
}
public StorageType[] getStorageTypes() {
return storageTypes;
}
public String[] getStorageIDs() {
return storageIDs;
}
public long getStartOffset() {
return offset;
}
@ -106,3 +141,4 @@ public String toString() {
+ "}";
}
}

View File

@ -51,7 +51,7 @@ public UnregisteredNodeException(NodeRegistration nodeReg) {
*/
public UnregisteredNodeException(DatanodeID nodeID, DatanodeInfo storedNode) {
super("Data node " + nodeID + " is attempting to report storage ID "
+ nodeID.getStorageID() + ". Node "
+ nodeID.getDatanodeUuid() + ". Node "
+ storedNode + " is expected to serve this storage.");
}
}

View File

@ -405,14 +405,17 @@ public GetAdditionalDatanodeResponseProto getAdditionalDatanode(
throws ServiceException {
try {
List<DatanodeInfoProto> existingList = req.getExistingsList();
List<String> existingStorageIDsList = req.getExistingStorageUuidsList();
List<DatanodeInfoProto> excludesList = req.getExcludesList();
LocatedBlock result = server.getAdditionalDatanode(
req.getSrc(), PBHelper.convert(req.getBlk()),
LocatedBlock result = server.getAdditionalDatanode(req.getSrc(),
PBHelper.convert(req.getBlk()),
PBHelper.convert(existingList.toArray(
new DatanodeInfoProto[existingList.size()])),
existingStorageIDsList.toArray(
new String[existingStorageIDsList.size()]),
PBHelper.convert(excludesList.toArray(
new DatanodeInfoProto[excludesList.size()])),
req.getNumAdditionalNodes(), req.getClientName());
req.getNumAdditionalNodes(), req.getClientName());
return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
PBHelper.convert(result))
.build();
@ -813,10 +816,12 @@ public UpdatePipelineResponseProto updatePipeline(RpcController controller,
UpdatePipelineRequestProto req) throws ServiceException {
try {
List<DatanodeIDProto> newNodes = req.getNewNodesList();
server
.updatePipeline(req.getClientName(), PBHelper.convert(req
.getOldBlock()), PBHelper.convert(req.getNewBlock()), PBHelper
.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])));
List<String> newStorageIDs = req.getStorageIDsList();
server.updatePipeline(req.getClientName(),
PBHelper.convert(req.getOldBlock()),
PBHelper.convert(req.getNewBlock()),
PBHelper.convert(newNodes.toArray(new DatanodeIDProto[newNodes.size()])),
newStorageIDs.toArray(new String[newStorageIDs.size()]));
return VOID_UPDATEPIPELINE_RESPONSE;
} catch (IOException e) {
throw new ServiceException(e);

View File

@ -335,7 +335,8 @@ public LocatedBlock addBlock(String src, String clientName,
@Override
public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
DatanodeInfo[] existings, DatanodeInfo[] excludes,
DatanodeInfo[] existings, String[] existingStorageIDs,
DatanodeInfo[] excludes,
int numAdditionalNodes, String clientName) throws AccessControlException,
FileNotFoundException, SafeModeException, UnresolvedLinkException,
IOException {
@ -344,6 +345,7 @@ public LocatedBlock getAdditionalDatanode(String src, ExtendedBlock blk,
.setSrc(src)
.setBlk(PBHelper.convert(blk))
.addAllExistings(PBHelper.convert(existings))
.addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
.addAllExcludes(PBHelper.convert(excludes))
.setNumAdditionalNodes(numAdditionalNodes)
.setClientName(clientName)
@ -780,12 +782,13 @@ public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
@Override
public void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes) throws IOException {
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] storageIDs) throws IOException {
UpdatePipelineRequestProto req = UpdatePipelineRequestProto.newBuilder()
.setClientName(clientName)
.setOldBlock(PBHelper.convert(oldBlock))
.setNewBlock(PBHelper.convert(newBlock))
.addAllNewNodes(Arrays.asList(PBHelper.convert(newNodes)))
.addAllStorageIDs(storageIDs == null ? null : Arrays.asList(storageIDs))
.build();
try {
rpcProxy.updatePipeline(null, req);

View File

@ -213,7 +213,7 @@ public void blockReceivedAndDeleted(DatanodeRegistration registration,
for (StorageReceivedDeletedBlocks storageBlock : receivedAndDeletedBlocks) {
StorageReceivedDeletedBlocksProto.Builder repBuilder =
StorageReceivedDeletedBlocksProto.newBuilder();
repBuilder.setStorageID(storageBlock.getStorageID());
repBuilder.setStorageUuid(storageBlock.getStorageID());
for (ReceivedDeletedBlockInfo rdBlock : storageBlock.getBlocks()) {
repBuilder.addBlocks(PBHelper.convert(rdBlock));
}

View File

@ -40,7 +40,6 @@
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlocksResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
@ -100,14 +99,8 @@ public HeartbeatResponseProto sendHeartbeat(RpcController controller,
HeartbeatRequestProto request) throws ServiceException {
HeartbeatResponse response;
try {
List<StorageReportProto> list = request.getReportsList();
StorageReport[] report = new StorageReport[list.size()];
int i = 0;
for (StorageReportProto p : list) {
report[i++] = new StorageReport(p.getStorageID(), p.getFailed(),
p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
p.getBlockPoolUsed());
}
final StorageReport[] report = PBHelper.convertStorageReports(
request.getReportsList());
response = impl.sendHeartbeat(PBHelper.convert(request.getRegistration()),
report, request.getXmitsInProgress(), request.getXceiverCount(),
request.getFailedVolumes());
@ -174,7 +167,7 @@ public BlockReceivedAndDeletedResponseProto blockReceivedAndDeleted(
for (int j = 0; j < list.size(); j++) {
rdBlocks[j] = PBHelper.convert(list.get(j));
}
info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageID(), rdBlocks);
info[i] = new StorageReceivedDeletedBlocks(sBlock.getStorageUuid(), rdBlocks);
}
try {
impl.blockReceivedAndDeleted(PBHelper.convert(request.getRegistration()),

View File

@ -82,6 +82,6 @@ public UpdateReplicaUnderRecoveryResponseProto updateReplicaUnderRecovery(
throw new ServiceException(e);
}
return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
.setStorageID(storageID).build();
.setStorageUuid(storageID).build();
}
}

View File

@ -109,7 +109,7 @@ public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
.setNewLength(newLength).setRecoveryId(recoveryId).build();
try {
return rpcProxy.updateReplicaUnderRecovery(NULL_CONTROLLER, req
).getStorageID();
).getStorageUuid();
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}

View File

@ -25,12 +25,14 @@
import java.util.EnumSet;
import java.util.List;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
@ -42,12 +44,12 @@
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos;
import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateFlagProto;
@ -104,6 +106,8 @@
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryStatusProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageInfoProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
import org.apache.hadoop.hdfs.security.token.block.BlockKey;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
@ -221,17 +225,20 @@ public static NamenodeRegistration convert(NamenodeRegistrationProto reg) {
// DatanodeId
public static DatanodeID convert(DatanodeIDProto dn) {
return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getStorageID(),
return new DatanodeID(dn.getIpAddr(), dn.getHostName(), dn.getDatanodeUuid(),
dn.getXferPort(), dn.getInfoPort(), dn.hasInfoSecurePort() ? dn
.getInfoSecurePort() : 0, dn.getIpcPort());
}
public static DatanodeIDProto convert(DatanodeID dn) {
// For wire compatibility with older versions we transmit the StorageID
// which is the same as the DatanodeUuid. Since StorageID is a required
// field we pass the empty string if the DatanodeUuid is not yet known.
return DatanodeIDProto.newBuilder()
.setIpAddr(dn.getIpAddr())
.setHostName(dn.getHostName())
.setStorageID(dn.getStorageID())
.setXferPort(dn.getXferPort())
.setDatanodeUuid(dn.getDatanodeUuid() != null ? dn.getDatanodeUuid() : "")
.setInfoPort(dn.getInfoPort())
.setInfoSecurePort(dn.getInfoSecurePort())
.setIpcPort(dn.getIpcPort()).build();
@ -273,12 +280,16 @@ public static Block convert(BlockProto b) {
public static BlockWithLocationsProto convert(BlockWithLocations blk) {
return BlockWithLocationsProto.newBuilder()
.setBlock(convert(blk.getBlock()))
.addAllStorageIDs(Arrays.asList(blk.getStorageIDs())).build();
.addAllDatanodeUuids(Arrays.asList(blk.getDatanodeUuids()))
.addAllStorageUuids(Arrays.asList(blk.getStorageIDs())).build();
}
public static BlockWithLocations convert(BlockWithLocationsProto b) {
return new BlockWithLocations(convert(b.getBlock()), b.getStorageIDsList()
.toArray(new String[0]));
final List<String> datanodeUuids = b.getDatanodeUuidsList();
final List<String> storageUuids = b.getStorageUuidsList();
return new BlockWithLocations(convert(b.getBlock()),
datanodeUuids.toArray(new String[datanodeUuids.size()]),
storageUuids.toArray(new String[storageUuids.size()]));
}
public static BlocksWithLocationsProto convert(BlocksWithLocations blks) {
@ -567,6 +578,18 @@ public static LocatedBlockProto convert(LocatedBlock b) {
for (int i = 0; i < locs.length; i++) {
builder.addLocs(i, PBHelper.convert(locs[i]));
}
StorageType[] storageTypes = b.getStorageTypes();
if (storageTypes != null) {
for (int i = 0; i < storageTypes.length; ++i) {
builder.addStorageTypes(PBHelper.convertStorageType(storageTypes[i]));
}
}
final String[] storageIDs = b.getStorageIDs();
if (storageIDs != null) {
builder.addAllStorageIDs(Arrays.asList(storageIDs));
}
return builder.setB(PBHelper.convert(b.getBlock()))
.setBlockToken(PBHelper.convert(b.getBlockToken()))
.setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
@ -579,8 +602,27 @@ public static LocatedBlock convert(LocatedBlockProto proto) {
for (int i = 0; i < locs.size(); i++) {
targets[i] = PBHelper.convert(locs.get(i));
}
final int storageTypesCount = proto.getStorageTypesCount();
final StorageType[] storageTypes;
if (storageTypesCount == 0) {
storageTypes = null;
} else {
Preconditions.checkState(storageTypesCount == locs.size());
storageTypes = convertStorageTypeProtos(proto.getStorageTypesList());
}
final int storageIDsCount = proto.getStorageIDsCount();
final String[] storageIDs;
if (storageIDsCount == 0) {
storageIDs = null;
} else {
Preconditions.checkState(storageIDsCount == locs.size());
storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
}
LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
proto.getOffset(), proto.getCorrupt());
storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt());
lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
return lb;
}
@ -719,7 +761,8 @@ public static BlockCommandProto convert(BlockCommand cmd) {
for (int i = 0; i < blocks.length; i++) {
builder.addBlocks(PBHelper.convert(blocks[i]));
}
builder.addAllTargets(PBHelper.convert(cmd.getTargets()));
builder.addAllTargets(convert(cmd.getTargets()))
.addAllTargetStorageUuids(convert(cmd.getTargetStorageIDs()));
return builder.build();
}
@ -732,6 +775,15 @@ private static List<DatanodeInfosProto> convert(DatanodeInfo[][] targets) {
return Arrays.asList(ret);
}
private static List<StorageUuidsProto> convert(String[][] targetStorageUuids) {
StorageUuidsProto[] ret = new StorageUuidsProto[targetStorageUuids.length];
for (int i = 0; i < targetStorageUuids.length; i++) {
ret[i] = StorageUuidsProto.newBuilder()
.addAllStorageUuids(Arrays.asList(targetStorageUuids[i])).build();
}
return Arrays.asList(ret);
}
public static DatanodeCommandProto convert(DatanodeCommand datanodeCommand) {
DatanodeCommandProto.Builder builder = DatanodeCommandProto.newBuilder();
if (datanodeCommand == null) {
@ -806,6 +858,14 @@ public static BlockCommand convert(BlockCommandProto blkCmd) {
for (int i = 0; i < targetList.size(); i++) {
targets[i] = PBHelper.convert(targetList.get(i));
}
List<StorageUuidsProto> targetStorageUuidsList = blkCmd.getTargetStorageUuidsList();
String[][] targetStorageIDs = new String[targetStorageUuidsList.size()][];
for(int i = 0; i < targetStorageIDs.length; i++) {
List<String> storageIDs = targetStorageUuidsList.get(i).getStorageUuidsList();
targetStorageIDs[i] = storageIDs.toArray(new String[storageIDs.size()]);
}
int action = DatanodeProtocol.DNA_UNKNOWN;
switch (blkCmd.getAction()) {
case TRANSFER:
@ -818,7 +878,8 @@ public static BlockCommand convert(BlockCommandProto blkCmd) {
action = DatanodeProtocol.DNA_SHUTDOWN;
break;
}
return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets);
return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets,
targetStorageIDs);
}
public static DatanodeInfo[] convert(DatanodeInfosProto datanodeInfosProto) {
@ -1328,11 +1389,12 @@ public static NNHAStatusHeartbeatProto convert(NNHAStatusHeartbeat hb) {
public static DatanodeStorageProto convert(DatanodeStorage s) {
return DatanodeStorageProto.newBuilder()
.setState(PBHelper.convert(s.getState()))
.setStorageID(s.getStorageID()).build();
.setState(PBHelper.convertState(s.getState()))
.setStorageType(PBHelper.convertStorageType(s.getStorageType()))
.setStorageUuid(s.getStorageID()).build();
}
private static StorageState convert(State state) {
private static StorageState convertState(State state) {
switch(state) {
case READ_ONLY:
return StorageState.READ_ONLY;
@ -1342,11 +1404,26 @@ private static StorageState convert(State state) {
}
}
public static DatanodeStorage convert(DatanodeStorageProto s) {
return new DatanodeStorage(s.getStorageID(), PBHelper.convert(s.getState()));
private static StorageTypeProto convertStorageType(
StorageType type) {
switch(type) {
case DISK:
return StorageTypeProto.DISK;
case SSD:
return StorageTypeProto.SSD;
default:
throw new IllegalStateException(
"BUG: StorageType not found, type=" + type);
}
}
private static State convert(StorageState state) {
public static DatanodeStorage convert(DatanodeStorageProto s) {
return new DatanodeStorage(s.getStorageUuid(),
PBHelper.convertState(s.getState()),
PBHelper.convertType(s.getStorageType()));
}
private static State convertState(StorageState state) {
switch(state) {
case READ_ONLY:
return DatanodeStorage.State.READ_ONLY;
@ -1356,11 +1433,47 @@ private static State convert(StorageState state) {
}
}
private static StorageType convertType(StorageTypeProto type) {
switch(type) {
case DISK:
return StorageType.DISK;
case SSD:
return StorageType.SSD;
default:
throw new IllegalStateException(
"BUG: StorageTypeProto not found, type=" + type);
}
}
private static StorageType[] convertStorageTypeProtos(
List<StorageTypeProto> storageTypesList) {
final StorageType[] storageTypes = new StorageType[storageTypesList.size()];
for (int i = 0; i < storageTypes.length; ++i) {
storageTypes[i] = PBHelper.convertType(storageTypesList.get(i));
}
return storageTypes;
}
public static StorageReportProto convert(StorageReport r) {
return StorageReportProto.newBuilder()
.setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
.setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
.setStorageID(r.getStorageID()).build();
.setStorageUuid(r.getStorageID()).build();
}
public static StorageReport convert(StorageReportProto p) {
return new StorageReport(p.getStorageUuid(), p.getFailed(),
p.getCapacity(), p.getDfsUsed(), p.getRemaining(),
p.getBlockPoolUsed());
}
public static StorageReport[] convertStorageReports(
List<StorageReportProto> list) {
final StorageReport[] report = new StorageReport[list.size()];
for (int i = 0; i < report.length; i++) {
report[i] = convert(list.get(i));
}
return report;
}
public static JournalInfo convert(JournalInfoProto info) {
@ -1497,3 +1610,4 @@ public static InputStream vintPrefixed(final InputStream input)
return new ExactSizeInputStream(input, size);
}
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hdfs.server.balancer;
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
import java.io.BufferedInputStream;
@ -221,9 +220,9 @@ public class Balancer {
private Map<Block, BalancerBlock> globalBlockList
= new HashMap<Block, BalancerBlock>();
private MovedBlocks movedBlocks = new MovedBlocks();
// Map storage IDs to BalancerDatanodes
private Map<String, BalancerDatanode> datanodes
= new HashMap<String, BalancerDatanode>();
/** Map (datanodeUuid -> BalancerDatanodes) */
private final Map<String, BalancerDatanode> datanodeMap
= new HashMap<String, BalancerDatanode>();
private NetworkTopology cluster;
@ -241,6 +240,14 @@ private class PendingBlockMove {
private PendingBlockMove() {
}
@Override
public String toString() {
final Block b = block.getBlock();
return b + " with size=" + b.getNumBytes() + " from "
+ source.getDisplayName() + " to " + target.getDisplayName()
+ " through " + proxySource.getDisplayName();
}
/* choose a block & a proxy source for this pendingMove
* whose source & target have already been chosen.
*
@ -272,11 +279,7 @@ private boolean markMovedIfGoodBlock(BalancerBlock block) {
if ( chooseProxySource() ) {
movedBlocks.add(block);
if (LOG.isDebugEnabled()) {
LOG.debug("Decided to move block "+ block.getBlockId()
+" with a length of "+StringUtils.byteDesc(block.getNumBytes())
+ " bytes from " + source.getDisplayName()
+ " to " + target.getDisplayName()
+ " using proxy source " + proxySource.getDisplayName() );
LOG.debug("Decided to move " + this);
}
return true;
}
@ -353,17 +356,9 @@ private void dispatch() {
sendRequest(out);
receiveResponse(in);
bytesMoved.inc(block.getNumBytes());
LOG.info( "Moving block " + block.getBlock().getBlockId() +
" from "+ source.getDisplayName() + " to " +
target.getDisplayName() + " through " +
proxySource.getDisplayName() +
" is succeeded." );
LOG.info("Successfully moved " + this);
} catch (IOException e) {
LOG.warn("Error moving block "+block.getBlockId()+
" from " + source.getDisplayName() + " to " +
target.getDisplayName() + " through " +
proxySource.getDisplayName() +
": "+e.getMessage());
LOG.warn("Failed to move " + this + ": " + e.getMessage());
} finally {
IOUtils.closeStream(out);
IOUtils.closeStream(in);
@ -415,9 +410,7 @@ private void scheduleBlockMove() {
@Override
public void run() {
if (LOG.isDebugEnabled()) {
LOG.debug("Starting moving "+ block.getBlockId() +
" from " + proxySource.getDisplayName() + " to " +
target.getDisplayName());
LOG.debug("Start moving " + PendingBlockMove.this);
}
dispatch();
}
@ -464,11 +457,6 @@ private Block getBlock() {
return block;
}
/* Return the block id */
private long getBlockId() {
return block.getBlockId();
}
/* Return the length of the block */
private long getNumBytes() {
return block.getNumBytes();
@ -552,7 +540,7 @@ protected String getDisplayName() {
/* Get the storage id of the datanode */
protected String getStorageID() {
return datanode.getStorageID();
return datanode.getDatanodeUuid();
}
/** Decide if still need to move more bytes */
@ -675,10 +663,10 @@ private long getBlockList() throws IOException {
synchronized (block) {
// update locations
for ( String storageID : blk.getStorageIDs() ) {
BalancerDatanode datanode = datanodes.get(storageID);
for (String datanodeUuid : blk.getDatanodeUuids()) {
final BalancerDatanode d = datanodeMap.get(datanodeUuid);
if (datanode != null) { // not an unknown datanode
block.addLocation(datanode);
block.addLocation(d);
}
}
}
@ -852,16 +840,6 @@ private static void checkReplicationPolicyCompatibility(Configuration conf
DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_DEFAULT));
}
/* Shuffle datanode array */
static private void shuffleArray(DatanodeInfo[] datanodes) {
for (int i=datanodes.length; i>1; i--) {
int randomIndex = DFSUtil.getRandom().nextInt(i);
DatanodeInfo tmp = datanodes[randomIndex];
datanodes[randomIndex] = datanodes[i-1];
datanodes[i-1] = tmp;
}
}
/* Given a data node set, build a network topology and decide
* over-utilized datanodes, above average utilized datanodes,
* below average utilized datanodes, and underutilized datanodes.
@ -891,8 +869,7 @@ private long initNodes(DatanodeInfo[] datanodes) {
* an increasing order or a decreasing order.
*/
long overLoadedBytes = 0L, underLoadedBytes = 0L;
shuffleArray(datanodes);
for (DatanodeInfo datanode : datanodes) {
for (DatanodeInfo datanode : DFSUtil.shuffle(datanodes)) {
if (datanode.isDecommissioned() || datanode.isDecommissionInProgress()) {
continue; // ignore decommissioning or decommissioned nodes
}
@ -923,13 +900,13 @@ assert isUnderUtilized(datanodeS) : "isUnderUtilized("
datanodeS.utilization)*datanodeS.datanode.getCapacity()/100.0);
}
}
this.datanodes.put(datanode.getStorageID(), datanodeS);
datanodeMap.put(datanode.getDatanodeUuid(), datanodeS);
}
//logging
logNodes();
assert (this.datanodes.size() ==
assert (this.datanodeMap.size() ==
overUtilizedDatanodes.size()+underUtilizedDatanodes.size()+
aboveAvgUtilizedDatanodes.size()+belowAvgUtilizedDatanodes.size())
: "Mismatched number of datanodes";
@ -1001,9 +978,9 @@ private long chooseNodes() {
// At last, match all remaining nodes
chooseNodes(ANY_OTHER);
assert (datanodes.size() >= sources.size()+targets.size())
assert (datanodeMap.size() >= sources.size()+targets.size())
: "Mismatched number of datanodes (" +
datanodes.size() + " total, " +
datanodeMap.size() + " total, " +
sources.size() + " sources, " +
targets.size() + " targets)";
@ -1304,7 +1281,7 @@ private void resetData(Configuration conf) {
this.aboveAvgUtilizedDatanodes.clear();
this.belowAvgUtilizedDatanodes.clear();
this.underUtilizedDatanodes.clear();
this.datanodes.clear();
this.datanodeMap.clear();
this.sources.clear();
this.targets.clear();
this.policy.reset();

View File

@ -21,6 +21,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.util.LightWeightGSet;
@ -42,11 +43,11 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
private LightWeightGSet.LinkedElement nextLinkedElement;
/**
* This array contains triplets of references. For each i-th datanode the
* block belongs to triplets[3*i] is the reference to the DatanodeDescriptor
* and triplets[3*i+1] and triplets[3*i+2] are references to the previous and
* the next blocks, respectively, in the list of blocks belonging to this
* data-node.
* This array contains triplets of references. For each i-th storage, the
* block belongs to triplets[3*i] is the reference to the
* {@link DatanodeStorageInfo} and triplets[3*i+1] and triplets[3*i+2] are
* references to the previous and the next blocks, respectively, in the list
* of blocks belonging to this storage.
*
* Using previous and next in Object triplets is done instead of a
* {@link LinkedList} list to efficiently use memory. With LinkedList the cost
@ -88,10 +89,15 @@ public void setBlockCollection(BlockCollection bc) {
this.bc = bc;
}
DatanodeDescriptor getDatanode(int index) {
public DatanodeDescriptor getDatanode(int index) {
DatanodeStorageInfo storage = getStorageInfo(index);
return storage == null ? null : storage.getDatanodeDescriptor();
}
DatanodeStorageInfo getStorageInfo(int index) {
assert this.triplets != null : "BlockInfo is not initialized";
assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
return (DatanodeDescriptor)triplets[index*3];
return (DatanodeStorageInfo)triplets[index*3];
}
private BlockInfo getPrevious(int index) {
@ -114,14 +120,10 @@ BlockInfo getNext(int index) {
return info;
}
private void setDatanode(int index, DatanodeDescriptor node, BlockInfo previous,
BlockInfo next) {
private void setStorageInfo(int index, DatanodeStorageInfo storage) {
assert this.triplets != null : "BlockInfo is not initialized";
int i = index * 3;
assert index >= 0 && i+2 < triplets.length : "Index is out of bound";
triplets[i] = node;
triplets[i+1] = previous;
triplets[i+2] = next;
assert index >= 0 && index*3 < triplets.length : "Index is out of bound";
triplets[index*3] = storage;
}
/**
@ -193,22 +195,34 @@ public int numNodes() {
}
/**
* Add data-node this block belongs to.
* Add a {@link DatanodeStorageInfo} location for a block
*/
public boolean addNode(DatanodeDescriptor node) {
if(findDatanode(node) >= 0) // the node is already there
return false;
boolean addStorage(DatanodeStorageInfo storage) {
boolean added = true;
int idx = findDatanode(storage.getDatanodeDescriptor());
if(idx >= 0) {
if (getStorageInfo(idx) == storage) { // the storage is already there
return false;
} else {
// The block is on the DN but belongs to a different storage.
// Update our state.
removeStorage(storage);
added = false; // Just updating storage. Return false.
}
}
// find the last null node
int lastNode = ensureCapacity(1);
setDatanode(lastNode, node, null, null);
return true;
setStorageInfo(lastNode, storage);
setNext(lastNode, null);
setPrevious(lastNode, null);
return added;
}
/**
* Remove data-node from the block.
* Remove {@link DatanodeStorageInfo} location for a block
*/
public boolean removeNode(DatanodeDescriptor node) {
int dnIndex = findDatanode(node);
boolean removeStorage(DatanodeStorageInfo storage) {
int dnIndex = findStorageInfo(storage);
if(dnIndex < 0) // the node is not found
return false;
assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
@ -216,10 +230,13 @@ assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
// find the last not null node
int lastNode = numNodes()-1;
// replace current node triplet by the lastNode one
setDatanode(dnIndex, getDatanode(lastNode), getPrevious(lastNode),
getNext(lastNode));
setStorageInfo(dnIndex, getStorageInfo(lastNode));
setNext(dnIndex, getNext(lastNode));
setPrevious(dnIndex, getPrevious(lastNode));
// set the last triplet to null
setDatanode(lastNode, null, null, null);
setStorageInfo(lastNode, null);
setNext(lastNode, null);
setPrevious(lastNode, null);
return true;
}
@ -239,37 +256,70 @@ int findDatanode(DatanodeDescriptor dn) {
}
return -1;
}
/**
* Find specified DatanodeStorageInfo.
* @param dn
* @return index or -1 if not found.
*/
int findStorageInfo(DatanodeInfo dn) {
int len = getCapacity();
for(int idx = 0; idx < len; idx++) {
DatanodeStorageInfo cur = getStorageInfo(idx);
if(cur == null)
break;
if(cur.getDatanodeDescriptor() == dn)
return idx;
}
return -1;
}
/**
* Find specified DatanodeStorageInfo.
* @param storageInfo
* @return index or -1 if not found.
*/
int findStorageInfo(DatanodeStorageInfo storageInfo) {
int len = getCapacity();
for(int idx = 0; idx < len; idx++) {
DatanodeStorageInfo cur = getStorageInfo(idx);
if(cur == storageInfo)
return idx;
if(cur == null)
break;
}
return -1;
}
/**
* Insert this block into the head of the list of blocks
* related to the specified DatanodeDescriptor.
* related to the specified DatanodeStorageInfo.
* 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) {
int dnIndex = this.findDatanode(dn);
BlockInfo listInsert(BlockInfo head, DatanodeStorageInfo storage) {
int dnIndex = this.findStorageInfo(storage);
assert dnIndex >= 0 : "Data node is not found: current";
assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
"Block is already in the list and cannot be inserted.";
this.setPrevious(dnIndex, null);
this.setNext(dnIndex, head);
if(head != null)
head.setPrevious(head.findDatanode(dn), this);
head.setPrevious(head.findStorageInfo(storage), this);
return this;
}
/**
* Remove this block from the list of blocks
* related to the specified DatanodeDescriptor.
* related to the specified DatanodeStorageInfo.
* If this block is the head of the list then return the next block as
* the new head.
* @return the new head of the list or null if the list becomes
* empty after deletion.
* empy after deletion.
*/
public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
BlockInfo listRemove(BlockInfo head, DatanodeStorageInfo storage) {
if(head == null)
return null;
int dnIndex = this.findDatanode(dn);
int dnIndex = this.findStorageInfo(storage);
if(dnIndex < 0) // this block is not on the data-node list
return head;
@ -278,9 +328,9 @@ public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
this.setNext(dnIndex, null);
this.setPrevious(dnIndex, null);
if(prev != null)
prev.setNext(prev.findDatanode(dn), next);
prev.setNext(prev.findStorageInfo(storage), next);
if(next != null)
next.setPrevious(next.findDatanode(dn), prev);
next.setPrevious(next.findStorageInfo(storage), prev);
if(this == head) // removing the head
head = next;
return head;
@ -292,7 +342,7 @@ public BlockInfo listRemove(BlockInfo head, DatanodeDescriptor dn) {
*
* @return the new head of the list.
*/
public BlockInfo moveBlockToHead(BlockInfo head, DatanodeDescriptor dn,
public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage,
int curIndex, int headIndex) {
if (head == this) {
return this;
@ -301,9 +351,9 @@ public BlockInfo moveBlockToHead(BlockInfo head, DatanodeDescriptor dn,
BlockInfo prev = this.setPrevious(curIndex, null);
head.setPrevious(headIndex, this);
prev.setNext(prev.findDatanode(dn), next);
prev.setNext(prev.findStorageInfo(storage), next);
if (next != null)
next.setPrevious(next.findDatanode(dn), prev);
next.setPrevious(next.findStorageInfo(storage), prev);
return this;
}
@ -331,10 +381,10 @@ public boolean isComplete() {
* @return BlockInfoUnderConstruction - an under construction block.
*/
public BlockInfoUnderConstruction convertToBlockUnderConstruction(
BlockUCState s, DatanodeDescriptor[] targets) {
BlockUCState s, DatanodeStorageInfo[] targets) {
if(isComplete()) {
return new BlockInfoUnderConstruction(
this, getBlockCollection().getBlockReplication(), s, targets);
return new BlockInfoUnderConstruction(this,
getBlockCollection().getBlockReplication(), s, targets);
}
// the block is already under construction
BlockInfoUnderConstruction ucBlock = (BlockInfoUnderConstruction)this;

View File

@ -63,12 +63,12 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* corresponding replicas.
*/
static class ReplicaUnderConstruction extends Block {
private DatanodeDescriptor expectedLocation;
private final DatanodeStorageInfo expectedLocation;
private ReplicaState state;
private boolean chosenAsPrimary;
ReplicaUnderConstruction(Block block,
DatanodeDescriptor target,
DatanodeStorageInfo target,
ReplicaState state) {
super(block);
this.expectedLocation = target;
@ -82,7 +82,7 @@ static class ReplicaUnderConstruction extends Block {
* It is not guaranteed, but expected, that the data-node actually has
* the replica.
*/
DatanodeDescriptor getExpectedLocation() {
private DatanodeStorageInfo getExpectedStorageLocation() {
return expectedLocation;
}
@ -118,7 +118,7 @@ void setChosenAsPrimary(boolean chosenAsPrimary) {
* Is data-node the replica belongs to alive.
*/
boolean isAlive() {
return expectedLocation.isAlive;
return expectedLocation.getDatanodeDescriptor().isAlive;
}
@Override // Block
@ -162,7 +162,7 @@ public BlockInfoUnderConstruction(Block blk, int replication) {
*/
public BlockInfoUnderConstruction(Block blk, int replication,
BlockUCState state,
DatanodeDescriptor[] targets) {
DatanodeStorageInfo[] targets) {
super(blk, replication);
assert getBlockUCState() != BlockUCState.COMPLETE :
"BlockInfoUnderConstruction cannot be in COMPLETE state";
@ -186,7 +186,7 @@ assert getBlockUCState() != BlockUCState.COMPLETE :
}
/** Set expected locations */
public void setExpectedLocations(DatanodeDescriptor[] targets) {
public void setExpectedLocations(DatanodeStorageInfo[] targets) {
int numLocations = targets == null ? 0 : targets.length;
this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
for(int i = 0; i < numLocations; i++)
@ -198,12 +198,12 @@ public void setExpectedLocations(DatanodeDescriptor[] targets) {
* Create array of expected replica locations
* (as has been assigned by chooseTargets()).
*/
public DatanodeDescriptor[] getExpectedLocations() {
public DatanodeStorageInfo[] getExpectedStorageLocations() {
int numLocations = replicas == null ? 0 : replicas.size();
DatanodeDescriptor[] locations = new DatanodeDescriptor[numLocations];
DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
for(int i = 0; i < numLocations; i++)
locations[i] = replicas.get(i).getExpectedLocation();
return locations;
storages[i] = replicas.get(i).getExpectedStorageLocation();
return storages;
}
/** Get the number of expected locations */
@ -244,9 +244,9 @@ public void setGenerationStampAndVerifyReplicas(long genStamp) {
// The replica list is unchanged.
for (ReplicaUnderConstruction r : replicas) {
if (genStamp != r.getGenerationStamp()) {
r.getExpectedLocation().removeBlock(this);
r.getExpectedStorageLocation().removeBlock(this);
NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
+ "from location: " + r.getExpectedLocation());
+ "from location: " + r.getExpectedStorageLocation());
}
}
}
@ -302,31 +302,44 @@ public void initializeBlockRecovery(long recoveryId) {
if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
continue;
}
if (replicas.get(i).getExpectedLocation().getLastUpdate() > mostRecentLastUpdate) {
primary = replicas.get(i);
final ReplicaUnderConstruction ruc = replicas.get(i);
final long lastUpdate = ruc.getExpectedStorageLocation().getDatanodeDescriptor().getLastUpdate();
if (lastUpdate > mostRecentLastUpdate) {
primaryNodeIndex = i;
mostRecentLastUpdate = primary.getExpectedLocation().getLastUpdate();
primary = ruc;
mostRecentLastUpdate = lastUpdate;
}
}
if (primary != null) {
primary.getExpectedLocation().addBlockToBeRecovered(this);
primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
primary.setChosenAsPrimary(true);
NameNode.blockStateChangeLog.info("BLOCK* " + this
+ " recovery started, primary=" + primary);
}
}
void addReplicaIfNotPresent(DatanodeDescriptor dn,
void addReplicaIfNotPresent(DatanodeStorageInfo storage,
Block block,
ReplicaState rState) {
for (ReplicaUnderConstruction r : replicas) {
if (r.getExpectedLocation() == dn) {
Iterator<ReplicaUnderConstruction> it = replicas.iterator();
while (it.hasNext()) {
ReplicaUnderConstruction r = it.next();
if(r.getExpectedStorageLocation() == storage) {
// Record the gen stamp from the report
r.setGenerationStamp(block.getGenerationStamp());
return;
} else if (r.getExpectedStorageLocation().getDatanodeDescriptor() ==
storage.getDatanodeDescriptor()) {
// The Datanode reported that the block is on a different storage
// than the one chosen by BlockPlacementPolicy. This can occur as
// we allow Datanodes to choose the target storage. Update our
// state by removing the stale entry and adding a new one.
it.remove();
break;
}
}
replicas.add(new ReplicaUnderConstruction(block, dn, rState));
replicas.add(new ReplicaUnderConstruction(block, storage, rState));
}
@Override // BlockInfo

View File

@ -34,6 +34,7 @@
import java.util.Queue;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
@ -44,6 +45,7 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportIterator;
@ -70,8 +72,10 @@
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.security.UserGroupInformation;
@ -470,8 +474,8 @@ public void metaSave(PrintWriter out) {
private void dumpBlockMeta(Block block, PrintWriter out) {
List<DatanodeDescriptor> containingNodes =
new ArrayList<DatanodeDescriptor>();
List<DatanodeDescriptor> containingLiveReplicasNodes =
new ArrayList<DatanodeDescriptor>();
List<DatanodeStorageInfo> containingLiveReplicasNodes =
new ArrayList<DatanodeStorageInfo>();
NumberReplicas numReplicas = new NumberReplicas();
// source node returned is not used
@ -498,9 +502,8 @@ private void dumpBlockMeta(Block block, PrintWriter out) {
Collection<DatanodeDescriptor> corruptNodes =
corruptReplicas.getNodes(block);
for (Iterator<DatanodeDescriptor> jt = blocksMap.nodeIterator(block);
jt.hasNext();) {
DatanodeDescriptor node = jt.next();
for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
String state = "";
if (corruptNodes != null && corruptNodes.contains(node)) {
state = "(corrupt)";
@ -509,7 +512,7 @@ private void dumpBlockMeta(Block block, PrintWriter out) {
state = "(decommissioned)";
}
if (node.areBlockContentsStale()) {
if (storage.areBlockContentsStale()) {
state += " (block deletions maybe out of date)";
}
out.print(" " + node + state + " : ");
@ -660,10 +663,9 @@ public LocatedBlock convertLastBlockToUnderConstruction(
assert oldBlock == getStoredBlock(oldBlock) :
"last block of the file is not in blocksMap";
DatanodeDescriptor[] targets = getNodes(oldBlock);
DatanodeStorageInfo[] targets = getStorages(oldBlock);
BlockInfoUnderConstruction ucBlock =
bc.setLastBlock(oldBlock, targets);
BlockInfoUnderConstruction ucBlock = bc.setLastBlock(oldBlock, targets);
blocksMap.replaceBlock(ucBlock);
// Remove block from replication queue.
@ -673,9 +675,8 @@ public LocatedBlock convertLastBlockToUnderConstruction(
pendingReplications.remove(ucBlock);
// remove this block from the list of pending blocks to be deleted.
for (DatanodeDescriptor dd : targets) {
String datanodeId = dd.getStorageID();
invalidateBlocks.remove(datanodeId, oldBlock);
for (DatanodeStorageInfo storage : targets) {
invalidateBlocks.remove(storage.getStorageID(), oldBlock);
}
// Adjust safe-mode totals, since under-construction blocks don't
@ -694,18 +695,17 @@ public LocatedBlock convertLastBlockToUnderConstruction(
/**
* Get all valid locations of the block
*/
private List<String> getValidLocations(Block block) {
ArrayList<String> machineSet =
new ArrayList<String>(blocksMap.numNodes(block));
for(Iterator<DatanodeDescriptor> it =
blocksMap.nodeIterator(block); it.hasNext();) {
String storageID = it.next().getStorageID();
private List<DatanodeStorageInfo> getValidLocations(Block block) {
final List<DatanodeStorageInfo> locations
= new ArrayList<DatanodeStorageInfo>(blocksMap.numNodes(block));
for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
final String storageID = storage.getStorageID();
// filter invalidate replicas
if(!invalidateBlocks.contains(storageID, block)) {
machineSet.add(storageID);
locations.add(storage);
}
}
return machineSet;
return locations;
}
private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
@ -773,9 +773,9 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
+ ", blk=" + blk);
}
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)blk;
final DatanodeDescriptor[] locations = uc.getExpectedLocations();
final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
return new LocatedBlock(eb, locations, pos, false);
return new LocatedBlock(eb, storages, pos, false);
}
// get block locations
@ -790,15 +790,14 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
final int numNodes = blocksMap.numNodes(blk);
final boolean isCorrupt = numCorruptNodes == numNodes;
final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes;
final DatanodeDescriptor[] machines = new DatanodeDescriptor[numMachines];
final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
int j = 0;
if (numMachines > 0) {
for(Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(blk);
it.hasNext();) {
final DatanodeDescriptor d = it.next();
for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
final DatanodeDescriptor d = storage.getDatanodeDescriptor();
final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
if (isCorrupt || (!isCorrupt && !replicaCorrupt))
machines[j++] = d;
machines[j++] = storage;
}
}
assert j == machines.length :
@ -990,13 +989,20 @@ void removeBlocksAssociatedTo(final DatanodeDescriptor node) {
}
node.resetBlocks();
invalidateBlocks.remove(node.getStorageID());
invalidateBlocks.remove(node.getDatanodeUuid());
// If the DN hasn't block-reported since the most recent
// failover, then we may have been holding up on processing
// over-replicated blocks because of it. But we can now
// process those blocks.
if (node.areBlockContentsStale()) {
boolean stale = false;
for(DatanodeStorageInfo storage : node.getStorageInfos()) {
if (storage.areBlockContentsStale()) {
stale = true;
break;
}
}
if (stale) {
rescanPostponedMisreplicatedBlocks();
}
}
@ -1015,9 +1021,8 @@ void addToInvalidates(final Block block, final DatanodeInfo datanode) {
*/
private void addToInvalidates(Block b) {
StringBuilder datanodes = new StringBuilder();
for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(b); it
.hasNext();) {
DatanodeDescriptor node = it.next();
for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
invalidateBlocks.add(b, node, false);
datanodes.append(node).append(" ");
}
@ -1035,7 +1040,7 @@ private void addToInvalidates(Block b) {
* for logging purposes
*/
public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
final DatanodeInfo dn, String reason) throws IOException {
final DatanodeInfo dn, String storageID, String reason) throws IOException {
assert namesystem.hasWriteLock();
final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
if (storedBlock == null) {
@ -1048,11 +1053,11 @@ public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
return;
}
markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason,
Reason.CORRUPTION_REPORTED), dn);
Reason.CORRUPTION_REPORTED), dn, storageID);
}
private void markBlockAsCorrupt(BlockToMarkCorrupt b,
DatanodeInfo dn) throws IOException {
DatanodeInfo dn, String storageID) throws IOException {
DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
if (node == null) {
throw new IOException("Cannot mark " + b
@ -1068,7 +1073,7 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b,
}
// Add replica to the data-node if it is not already there
node.addBlock(b.stored);
node.addBlock(storageID, b.stored);
// Add this replica to corruptReplicas Map
corruptReplicas.addToCorruptReplicasMap(b.corrupted, node, b.reason,
@ -1193,7 +1198,7 @@ int computeReplicationWork(int blocksToProcess) {
@VisibleForTesting
int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
int requiredReplication, numEffectiveReplicas;
List<DatanodeDescriptor> containingNodes, liveReplicaNodes;
List<DatanodeDescriptor> containingNodes;
DatanodeDescriptor srcNode;
BlockCollection bc = null;
int additionalReplRequired;
@ -1219,7 +1224,7 @@ int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
// get a source data-node
containingNodes = new ArrayList<DatanodeDescriptor>();
liveReplicaNodes = new ArrayList<DatanodeDescriptor>();
List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<DatanodeStorageInfo>();
NumberReplicas numReplicas = new NumberReplicas();
srcNode = chooseSourceDatanode(
block, containingNodes, liveReplicaNodes, numReplicas,
@ -1279,7 +1284,7 @@ int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
namesystem.writeLock();
try {
for(ReplicationWork rw : work){
DatanodeDescriptor[] targets = rw.targets;
final DatanodeStorageInfo[] targets = rw.targets;
if(targets == null || targets.length == 0){
rw.targets = null;
continue;
@ -1319,7 +1324,8 @@ int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
if ( (numReplicas.liveReplicas() >= requiredReplication) &&
(!blockHasEnoughRacks(block)) ) {
if (rw.srcNode.getNetworkLocation().equals(targets[0].getNetworkLocation())) {
if (rw.srcNode.getNetworkLocation().equals(
targets[0].getDatanodeDescriptor().getNetworkLocation())) {
//No use continuing, unless a new rack in this case
continue;
}
@ -1328,15 +1334,13 @@ int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
// Add block to the to be replicated list
rw.srcNode.addBlockToBeReplicated(block, targets);
scheduledWork++;
for (DatanodeDescriptor dn : targets) {
dn.incBlocksScheduled();
}
DatanodeStorageInfo.incrementBlocksScheduled(targets);
// Move the block-replication into a "pending" state.
// The reason we use 'pending' is so we can retry
// replications that fail after an appropriate amount of time.
pendingReplications.increment(block, targets);
pendingReplications.increment(block,
DatanodeStorageInfo.toDatanodeDescriptors(targets));
if(blockLog.isDebugEnabled()) {
blockLog.debug(
"BLOCK* block " + block
@ -1357,12 +1361,12 @@ int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
if (blockLog.isInfoEnabled()) {
// log which blocks have been scheduled for replication
for(ReplicationWork rw : work){
DatanodeDescriptor[] targets = rw.targets;
DatanodeStorageInfo[] targets = rw.targets;
if (targets != null && targets.length != 0) {
StringBuilder targetList = new StringBuilder("datanode(s)");
for (int k = 0; k < targets.length; k++) {
targetList.append(' ');
targetList.append(targets[k]);
targetList.append(targets[k].getDatanodeDescriptor());
}
blockLog.info("BLOCK* ask " + rw.srcNode
+ " to replicate " + rw.block + " to " + targetList);
@ -1386,15 +1390,16 @@ int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
* @see BlockPlacementPolicy#chooseTarget(String, int, Node,
* List, boolean, Set, long)
*/
public DatanodeDescriptor[] chooseTarget(final String src,
public DatanodeStorageInfo[] chooseTarget(final String src,
final int numOfReplicas, final DatanodeDescriptor client,
final Set<Node> excludedNodes,
final long blocksize, List<String> favoredNodes) throws IOException {
List<DatanodeDescriptor> favoredDatanodeDescriptors =
getDatanodeDescriptors(favoredNodes);
final DatanodeDescriptor targets[] = blockplacement.chooseTarget(src,
final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
numOfReplicas, client, excludedNodes, blocksize,
favoredDatanodeDescriptors);
// TODO: get storage type from file
favoredDatanodeDescriptors, StorageType.DEFAULT);
if (targets.length < minReplication) {
throw new IOException("File " + src + " could only be replicated to "
+ targets.length + " nodes instead of minReplication (="
@ -1455,12 +1460,11 @@ List<DatanodeDescriptor> getDatanodeDescriptors(List<String> nodes) {
* the given block
*/
@VisibleForTesting
DatanodeDescriptor chooseSourceDatanode(
Block block,
List<DatanodeDescriptor> containingNodes,
List<DatanodeDescriptor> nodesContainingLiveReplicas,
NumberReplicas numReplicas,
int priority) {
DatanodeDescriptor chooseSourceDatanode(Block block,
List<DatanodeDescriptor> containingNodes,
List<DatanodeStorageInfo> nodesContainingLiveReplicas,
NumberReplicas numReplicas,
int priority) {
containingNodes.clear();
nodesContainingLiveReplicas.clear();
DatanodeDescriptor srcNode = null;
@ -1468,12 +1472,12 @@ DatanodeDescriptor chooseSourceDatanode(
int decommissioned = 0;
int corrupt = 0;
int excess = 0;
Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
while(it.hasNext()) {
DatanodeDescriptor node = it.next();
for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
LightWeightLinkedSet<Block> excessBlocks =
excessReplicateMap.get(node.getStorageID());
excessReplicateMap.get(node.getDatanodeUuid());
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
corrupt++;
else if (node.isDecommissionInProgress() || node.isDecommissioned())
@ -1481,7 +1485,7 @@ else if (node.isDecommissionInProgress() || node.isDecommissioned())
else if (excessBlocks != null && excessBlocks.contains(block)) {
excess++;
} else {
nodesContainingLiveReplicas.add(node);
nodesContainingLiveReplicas.add(storage);
live++;
}
containingNodes.add(node);
@ -1613,10 +1617,11 @@ public String toString() {
}
/**
* The given datanode is reporting all its blocks.
* Update the (machine-->blocklist) and (block-->machinelist) maps.
* The given storage is reporting all its blocks.
* Update the (storage-->block list) and (block-->storage list) maps.
*/
public void processReport(final DatanodeID nodeID, final String poolId,
public void processReport(final DatanodeID nodeID,
final DatanodeStorage storage, final String poolId,
final BlockListAsLongs newReport) throws IOException {
namesystem.writeLock();
final long startTime = Time.now(); //after acquiring write lock
@ -1630,26 +1635,28 @@ public void processReport(final DatanodeID nodeID, final String poolId,
// To minimize startup time, we discard any second (or later) block reports
// that we receive while still in startup phase.
if (namesystem.isInStartupSafeMode() && !node.isFirstBlockReport()) {
final DatanodeStorageInfo storageInfo = node.updateStorage(storage);
if (namesystem.isInStartupSafeMode()
&& storageInfo.getBlockReportCount() > 0) {
blockLog.info("BLOCK* processReport: "
+ "discarded non-initial block report from " + nodeID
+ " because namenode still in startup phase");
return;
}
if (node.numBlocks() == 0) {
if (storageInfo.numBlocks() == 0) {
// The first block report can be processed a lot more efficiently than
// ordinary block reports. This shortens restart times.
processFirstBlockReport(node, newReport);
processFirstBlockReport(node, storage.getStorageID(), newReport);
} else {
processReport(node, newReport);
processReport(node, storage, newReport);
}
// Now that we have an up-to-date block report, we know that any
// deletions from a previous NN iteration have been accounted for.
boolean staleBefore = node.areBlockContentsStale();
node.receivedBlockReport();
if (staleBefore && !node.areBlockContentsStale()) {
boolean staleBefore = storageInfo.areBlockContentsStale();
storageInfo.receivedBlockReport();
if (staleBefore && !storageInfo.areBlockContentsStale()) {
LOG.info("BLOCK* processReport: Received first block report from "
+ node + " after starting up or becoming active. Its block "
+ "contents are no longer considered stale");
@ -1703,28 +1710,30 @@ private void rescanPostponedMisreplicatedBlocks() {
}
private void processReport(final DatanodeDescriptor node,
final DatanodeStorage storage,
final BlockListAsLongs report) throws IOException {
// Normal case:
// Modify the (block-->datanode) map, according to the difference
// between the old and new block report.
//
Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
Collection<Block> toRemove = new LinkedList<Block>();
Collection<Block> toRemove = new TreeSet<Block>();
Collection<Block> toInvalidate = new LinkedList<Block>();
Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
reportDiff(node, report, toAdd, toRemove, toInvalidate, toCorrupt, toUC);
reportDiff(node, storage, report,
toAdd, toRemove, toInvalidate, toCorrupt, toUC);
// Process the blocks on each queue
for (StatefulBlockInfo b : toUC) {
addStoredBlockUnderConstruction(b, node);
addStoredBlockUnderConstruction(b, node, storage.getStorageID());
}
for (Block b : toRemove) {
removeStoredBlock(b, node);
}
int numBlocksLogged = 0;
for (BlockInfo b : toAdd) {
addStoredBlock(b, node, null, numBlocksLogged < maxNumBlocksToLog);
addStoredBlock(b, node, storage.getStorageID(), null, numBlocksLogged < maxNumBlocksToLog);
numBlocksLogged++;
}
if (numBlocksLogged > maxNumBlocksToLog) {
@ -1738,7 +1747,7 @@ private void processReport(final DatanodeDescriptor node,
addToInvalidates(b, node);
}
for (BlockToMarkCorrupt b : toCorrupt) {
markBlockAsCorrupt(b, node);
markBlockAsCorrupt(b, node, storage.getStorageID());
}
}
@ -1754,10 +1763,11 @@ private void processReport(final DatanodeDescriptor node,
* @throws IOException
*/
private void processFirstBlockReport(final DatanodeDescriptor node,
final String storageID,
final BlockListAsLongs report) throws IOException {
if (report == null) return;
assert (namesystem.hasWriteLock());
assert (node.numBlocks() == 0);
assert (node.getStorageInfo(storageID).numBlocks() == 0);
BlockReportIterator itBR = report.getBlockReportIterator();
while(itBR.hasNext()) {
@ -1766,7 +1776,7 @@ private void processFirstBlockReport(final DatanodeDescriptor node,
if (shouldPostponeBlocksFromFuture &&
namesystem.isGenStampInFuture(iblk)) {
queueReportedBlock(node, iblk, reportedState,
queueReportedBlock(node, storageID, iblk, reportedState,
QUEUE_REASON_FUTURE_GENSTAMP);
continue;
}
@ -1783,10 +1793,10 @@ private void processFirstBlockReport(final DatanodeDescriptor node,
if (shouldPostponeBlocksFromFuture) {
// In the Standby, we may receive a block report for a file that we
// just have an out-of-date gen-stamp or state for, for example.
queueReportedBlock(node, iblk, reportedState,
queueReportedBlock(node, storageID, iblk, reportedState,
QUEUE_REASON_CORRUPT_STATE);
} else {
markBlockAsCorrupt(c, node);
markBlockAsCorrupt(c, node, storageID);
}
continue;
}
@ -1794,7 +1804,7 @@ private void processFirstBlockReport(final DatanodeDescriptor node,
// If block is under construction, add this replica to its list
if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
((BlockInfoUnderConstruction)storedBlock).addReplicaIfNotPresent(
node, iblk, reportedState);
node.getStorageInfo(storageID), iblk, reportedState);
// OpenFileBlocks only inside snapshots also will be added to safemode
// threshold. So we need to update such blocks to safemode
// refer HDFS-5283
@ -1807,22 +1817,25 @@ private void processFirstBlockReport(final DatanodeDescriptor node,
}
//add replica if appropriate
if (reportedState == ReplicaState.FINALIZED) {
addStoredBlockImmediate(storedBlock, node);
addStoredBlockImmediate(storedBlock, node, storageID);
}
}
}
private void reportDiff(DatanodeDescriptor dn,
private void reportDiff(DatanodeDescriptor dn, DatanodeStorage storage,
BlockListAsLongs newReport,
Collection<BlockInfo> toAdd, // add to DatanodeDescriptor
Collection<Block> toRemove, // remove from DatanodeDescriptor
Collection<Block> toInvalidate, // should be removed from DN
Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
Collection<StatefulBlockInfo> toUC) { // add to under-construction list
final DatanodeStorageInfo storageInfo = dn.updateStorage(storage);
// place a delimiter in the list which separates blocks
// that have been reported from those that have not
BlockInfo delimiter = new BlockInfo(new Block(), 1);
boolean added = dn.addBlock(delimiter);
boolean added = storageInfo.addBlock(delimiter);
assert added : "Delimiting block cannot be present in the node";
int headIndex = 0; //currently the delimiter is in the head of the list
int curIndex;
@ -1834,20 +1847,21 @@ private void reportDiff(DatanodeDescriptor dn,
while(itBR.hasNext()) {
Block iblk = itBR.next();
ReplicaState iState = itBR.getCurrentReplicaState();
BlockInfo storedBlock = processReportedBlock(dn, iblk, iState,
toAdd, toInvalidate, toCorrupt, toUC);
BlockInfo storedBlock = processReportedBlock(dn, storage.getStorageID(),
iblk, iState, toAdd, toInvalidate, toCorrupt, toUC);
// move block to the head of the list
if (storedBlock != null && (curIndex = storedBlock.findDatanode(dn)) >= 0) {
headIndex = dn.moveBlockToHead(storedBlock, curIndex, headIndex);
headIndex = storageInfo.moveBlockToHead(storedBlock, curIndex, headIndex);
}
}
// collect blocks that have not been reported
// all of them are next to the delimiter
Iterator<? extends Block> it = new DatanodeDescriptor.BlockIterator(
delimiter.getNext(0), dn);
Iterator<BlockInfo> it = storageInfo.new BlockIterator(delimiter.getNext(0));
while(it.hasNext())
toRemove.add(it.next());
dn.removeBlock(delimiter);
storageInfo.removeBlock(delimiter);
}
/**
@ -1881,7 +1895,8 @@ private void reportDiff(DatanodeDescriptor dn,
* @return the up-to-date stored block, if it should be kept.
* Otherwise, null.
*/
private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
final String storageID,
final Block block, final ReplicaState reportedState,
final Collection<BlockInfo> toAdd,
final Collection<Block> toInvalidate,
@ -1896,7 +1911,7 @@ private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
if (shouldPostponeBlocksFromFuture &&
namesystem.isGenStampInFuture(block)) {
queueReportedBlock(dn, block, reportedState,
queueReportedBlock(dn, storageID, block, reportedState,
QUEUE_REASON_FUTURE_GENSTAMP);
return null;
}
@ -1917,7 +1932,7 @@ private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
}
// Ignore replicas already scheduled to be removed from the DN
if(invalidateBlocks.contains(dn.getStorageID(), block)) {
if(invalidateBlocks.contains(dn.getDatanodeUuid(), block)) {
/* TODO: following assertion is incorrect, see HDFS-2668
assert storedBlock.findDatanode(dn) < 0 : "Block " + block
+ " in recentInvalidatesSet should not appear in DN " + dn; */
@ -1931,7 +1946,7 @@ private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
// If the block is an out-of-date generation stamp or state,
// but we're the standby, we shouldn't treat it as corrupt,
// but instead just queue it for later processing.
queueReportedBlock(dn, storedBlock, reportedState,
queueReportedBlock(dn, storageID, storedBlock, reportedState,
QUEUE_REASON_CORRUPT_STATE);
} else {
toCorrupt.add(c);
@ -1960,7 +1975,7 @@ private BlockInfo processReportedBlock(final DatanodeDescriptor dn,
* standby node. @see PendingDataNodeMessages.
* @param reason a textual reason to report in the debug logs
*/
private void queueReportedBlock(DatanodeDescriptor dn, Block block,
private void queueReportedBlock(DatanodeDescriptor dn, String storageID, Block block,
ReplicaState reportedState, String reason) {
assert shouldPostponeBlocksFromFuture;
@ -1970,7 +1985,7 @@ private void queueReportedBlock(DatanodeDescriptor dn, Block block,
" from datanode " + dn + " for later processing " +
"because " + reason + ".");
}
pendingDNMessages.enqueueReportedBlock(dn, block, reportedState);
pendingDNMessages.enqueueReportedBlock(dn, storageID, block, reportedState);
}
/**
@ -1993,8 +2008,8 @@ private void processQueuedMessages(Iterable<ReportedBlockInfo> rbis)
if (LOG.isDebugEnabled()) {
LOG.debug("Processing previouly queued message " + rbi);
}
processAndHandleReportedBlock(
rbi.getNode(), rbi.getBlock(), rbi.getReportedState(), null);
processAndHandleReportedBlock(rbi.getNode(), rbi.getStorageID(),
rbi.getBlock(), rbi.getReportedState(), null);
}
}
@ -2111,19 +2126,21 @@ private boolean isBlockUnderConstruction(BlockInfo storedBlock,
return false;
}
}
void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
DatanodeDescriptor node) throws IOException {
DatanodeDescriptor node, String storageID) throws IOException {
BlockInfoUnderConstruction block = ucBlock.storedBlock;
block.addReplicaIfNotPresent(node, ucBlock.reportedBlock, ucBlock.reportedState);
block.addReplicaIfNotPresent(node.getStorageInfo(storageID),
ucBlock.reportedBlock, ucBlock.reportedState);
if (ucBlock.reportedState == ReplicaState.FINALIZED && block.findDatanode(node) < 0) {
addStoredBlock(block, node, null, true);
addStoredBlock(block, node, storageID, null, true);
}
}
}
/**
* Faster version of
* {@link #addStoredBlock(BlockInfo, DatanodeDescriptor, DatanodeDescriptor, boolean)}
* {@link #addStoredBlock(BlockInfo, DatanodeDescriptor, String, DatanodeDescriptor, boolean)}
* , intended for use with initial block report at startup. If not in startup
* safe mode, will call standard addStoredBlock(). Assumes this method is
* called "immediately" so there is no need to refresh the storedBlock from
@ -2134,17 +2151,17 @@ void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
* @throws IOException
*/
private void addStoredBlockImmediate(BlockInfo storedBlock,
DatanodeDescriptor node)
DatanodeDescriptor node, String storageID)
throws IOException {
assert (storedBlock != null && namesystem.hasWriteLock());
if (!namesystem.isInStartupSafeMode()
|| namesystem.isPopulatingReplQueues()) {
addStoredBlock(storedBlock, node, null, false);
addStoredBlock(storedBlock, node, storageID, null, false);
return;
}
// just add it
node.addBlock(storedBlock);
node.addBlock(storageID, storedBlock);
// Now check for completion of blocks and safe block count
int numCurrentReplica = countLiveNodes(storedBlock);
@ -2167,6 +2184,7 @@ private void addStoredBlockImmediate(BlockInfo storedBlock,
*/
private Block addStoredBlock(final BlockInfo block,
DatanodeDescriptor node,
String storageID,
DatanodeDescriptor delNodeHint,
boolean logEveryBlock)
throws IOException {
@ -2192,7 +2210,7 @@ private Block addStoredBlock(final BlockInfo block,
assert bc != null : "Block must belong to a file";
// add block to the datanode
boolean added = node.addBlock(storedBlock);
boolean added = node.addBlock(storageID, storedBlock);
int curReplicaDelta;
if (added) {
@ -2452,19 +2470,19 @@ private void processOverReplicatedBlock(final Block block,
Collection<DatanodeDescriptor> nonExcess = new ArrayList<DatanodeDescriptor>();
Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
.getNodes(block);
for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
it.hasNext();) {
DatanodeDescriptor cur = it.next();
if (cur.areBlockContentsStale()) {
for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
if (storage.areBlockContentsStale()) {
LOG.info("BLOCK* processOverReplicatedBlock: " +
"Postponing processing of over-replicated " +
block + " since datanode " + cur + " does not yet have up-to-date " +
block + " since storage + " + storage
+ "datanode " + cur + " does not yet have up-to-date " +
"block information.");
postponeBlock(block);
return;
}
LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(cur
.getStorageID());
.getDatanodeUuid());
if (excessBlocks == null || !excessBlocks.contains(block)) {
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
// exclude corrupt replicas
@ -2553,10 +2571,10 @@ private void chooseExcessReplicates(Collection<DatanodeDescriptor> nonExcess,
private void addToExcessReplicate(DatanodeInfo dn, Block block) {
assert namesystem.hasWriteLock();
LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getStorageID());
LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid());
if (excessBlocks == null) {
excessBlocks = new LightWeightLinkedSet<Block>();
excessReplicateMap.put(dn.getStorageID(), excessBlocks);
excessReplicateMap.put(dn.getDatanodeUuid(), excessBlocks);
}
if (excessBlocks.add(block)) {
excessBlocksCount.incrementAndGet();
@ -2604,7 +2622,7 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) {
// in "excess" there.
//
LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(node
.getStorageID());
.getDatanodeUuid());
if (excessBlocks != null) {
if (excessBlocks.remove(block)) {
excessBlocksCount.decrementAndGet();
@ -2613,7 +2631,7 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) {
+ block + " is removed from excessBlocks");
}
if (excessBlocks.size() == 0) {
excessReplicateMap.remove(node.getStorageID());
excessReplicateMap.remove(node.getDatanodeUuid());
}
}
}
@ -2628,12 +2646,18 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) {
* return the length of the added block; 0 if the block is not added
*/
private long addBlock(Block block, List<BlockWithLocations> results) {
final List<String> machineSet = getValidLocations(block);
if(machineSet.size() == 0) {
final List<DatanodeStorageInfo> locations = getValidLocations(block);
if(locations.size() == 0) {
return 0;
} else {
results.add(new BlockWithLocations(block,
machineSet.toArray(new String[machineSet.size()])));
final String[] datanodeUuids = new String[locations.size()];
final String[] storageIDs = new String[datanodeUuids.length];
for(int i = 0; i < locations.size(); i++) {
final DatanodeStorageInfo s = locations.get(i);
datanodeUuids[i] = s.getDatanodeDescriptor().getDatanodeUuid();
storageIDs[i] = s.getStorageID();
}
results.add(new BlockWithLocations(block, datanodeUuids, storageIDs));
return block.getNumBytes();
}
}
@ -2642,12 +2666,12 @@ private long addBlock(Block block, List<BlockWithLocations> results) {
* The given node is reporting that it received a certain block.
*/
@VisibleForTesting
void addBlock(DatanodeDescriptor node, Block block, String delHint)
void addBlock(DatanodeDescriptor node, String storageID, Block block, String delHint)
throws IOException {
// decrement number of blocks scheduled to this datanode.
// Decrement number of blocks scheduled to this datanode.
// for a retry request (of DatanodeProtocol#blockReceivedAndDeleted with
// RECEIVED_BLOCK), we currently also decrease the approximate number.
node.decBlocksScheduled();
node.decrementBlocksScheduled();
// get the deletion hint node
DatanodeDescriptor delHintNode = null;
@ -2663,11 +2687,12 @@ void addBlock(DatanodeDescriptor node, Block block, String delHint)
// Modify the blocks->datanode map and node's map.
//
pendingReplications.decrement(block, node);
processAndHandleReportedBlock(node, block, ReplicaState.FINALIZED,
processAndHandleReportedBlock(node, storageID, block, ReplicaState.FINALIZED,
delHintNode);
}
private void processAndHandleReportedBlock(DatanodeDescriptor node, Block block,
private void processAndHandleReportedBlock(DatanodeDescriptor node,
String storageID, Block block,
ReplicaState reportedState, DatanodeDescriptor delHintNode)
throws IOException {
// blockReceived reports a finalized block
@ -2675,7 +2700,7 @@ private void processAndHandleReportedBlock(DatanodeDescriptor node, Block block,
Collection<Block> toInvalidate = new LinkedList<Block>();
Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
processReportedBlock(node, block, reportedState,
processReportedBlock(node, storageID, block, reportedState,
toAdd, toInvalidate, toCorrupt, toUC);
// the block is only in one of the to-do lists
// if it is in none then data-node already has it
@ -2683,11 +2708,11 @@ private void processAndHandleReportedBlock(DatanodeDescriptor node, Block block,
: "The block should be only in one of the lists.";
for (StatefulBlockInfo b : toUC) {
addStoredBlockUnderConstruction(b, node);
addStoredBlockUnderConstruction(b, node, storageID);
}
long numBlocksLogged = 0;
for (BlockInfo b : toAdd) {
addStoredBlock(b, node, delHintNode, numBlocksLogged < maxNumBlocksToLog);
addStoredBlock(b, node, storageID, delHintNode, numBlocksLogged < maxNumBlocksToLog);
numBlocksLogged++;
}
if (numBlocksLogged > maxNumBlocksToLog) {
@ -2701,7 +2726,7 @@ private void processAndHandleReportedBlock(DatanodeDescriptor node, Block block,
addToInvalidates(b, node);
}
for (BlockToMarkCorrupt b : toCorrupt) {
markBlockAsCorrupt(b, node);
markBlockAsCorrupt(b, node, storageID);
}
}
@ -2713,7 +2738,7 @@ private void processAndHandleReportedBlock(DatanodeDescriptor node, Block block,
* This method must be called with FSNamesystem lock held.
*/
public void processIncrementalBlockReport(final DatanodeID nodeID,
final String poolId, final ReceivedDeletedBlockInfo blockInfos[])
final String poolId, final StorageReceivedDeletedBlocks srdb)
throws IOException {
assert namesystem.hasWriteLock();
int received = 0;
@ -2729,19 +2754,19 @@ public void processIncrementalBlockReport(final DatanodeID nodeID,
"Got incremental block report from unregistered or dead node");
}
for (ReceivedDeletedBlockInfo rdbi : blockInfos) {
for (ReceivedDeletedBlockInfo rdbi : srdb.getBlocks()) {
switch (rdbi.getStatus()) {
case DELETED_BLOCK:
removeStoredBlock(rdbi.getBlock(), node);
deleted++;
break;
case RECEIVED_BLOCK:
addBlock(node, rdbi.getBlock(), rdbi.getDelHints());
addBlock(node, srdb.getStorageID(), rdbi.getBlock(), rdbi.getDelHints());
received++;
break;
case RECEIVING_BLOCK:
receiving++;
processAndHandleReportedBlock(node, rdbi.getBlock(),
processAndHandleReportedBlock(node, srdb.getStorageID(), rdbi.getBlock(),
ReplicaState.RBW, null);
break;
default:
@ -2773,24 +2798,23 @@ public NumberReplicas countNodes(Block b) {
int corrupt = 0;
int excess = 0;
int stale = 0;
Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(b);
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
while (nodeIter.hasNext()) {
DatanodeDescriptor node = nodeIter.next();
for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
if ((nodesCorrupt != null) && (nodesCorrupt.contains(node))) {
corrupt++;
} else if (node.isDecommissionInProgress() || node.isDecommissioned()) {
decommissioned++;
} else {
LightWeightLinkedSet<Block> blocksExcess = excessReplicateMap.get(node
.getStorageID());
.getDatanodeUuid());
if (blocksExcess != null && blocksExcess.contains(b)) {
excess++;
} else {
live++;
}
}
if (node.areBlockContentsStale()) {
if (storage.areBlockContentsStale()) {
stale++;
}
}
@ -2813,10 +2837,9 @@ int countLiveNodes(BlockInfo b) {
}
// else proceed with fast case
int live = 0;
Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(b);
Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(b);
while (nodeIter.hasNext()) {
DatanodeDescriptor node = nodeIter.next();
for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
if ((nodesCorrupt == null) || (!nodesCorrupt.contains(node)))
live++;
}
@ -2828,10 +2851,9 @@ private void logBlockReplicationInfo(Block block, DatanodeDescriptor srcNode,
int curReplicas = num.liveReplicas();
int curExpectedReplicas = getReplication(block);
BlockCollection bc = blocksMap.getBlockCollection(block);
Iterator<DatanodeDescriptor> nodeIter = blocksMap.nodeIterator(block);
StringBuilder nodeList = new StringBuilder();
while (nodeIter.hasNext()) {
DatanodeDescriptor node = nodeIter.next();
for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
nodeList.append(node);
nodeList.append(" ");
}
@ -2936,14 +2958,13 @@ public int getActiveBlockCount() {
return blocksMap.size();
}
public DatanodeDescriptor[] getNodes(BlockInfo block) {
DatanodeDescriptor[] nodes =
new DatanodeDescriptor[block.numNodes()];
Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(block);
for (int i = 0; it != null && it.hasNext(); i++) {
nodes[i] = it.next();
public DatanodeStorageInfo[] getStorages(BlockInfo block) {
final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
int i = 0;
for(DatanodeStorageInfo s : blocksMap.getStorages(block)) {
storages[i++] = s;
}
return nodes;
return storages;
}
public int getTotalBlocks() {
@ -3056,9 +3077,8 @@ boolean blockHasEnoughRacks(Block b) {
corruptReplicas.getNodes(b);
int numExpectedReplicas = getReplication(b);
String rackName = null;
for (Iterator<DatanodeDescriptor> it = blocksMap.nodeIterator(b);
it.hasNext();) {
DatanodeDescriptor cur = it.next();
for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
if (numExpectedReplicas == 1 ||
@ -3102,8 +3122,8 @@ public BlockCollection getBlockCollection(Block b) {
}
/** @return an iterator of the datanodes. */
public Iterator<DatanodeDescriptor> datanodeIterator(final Block block) {
return blocksMap.nodeIterator(block);
public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
return blocksMap.getStorages(block);
}
public int numCorruptReplicas(Block block) {
@ -3247,24 +3267,24 @@ private static class ReplicationWork {
private DatanodeDescriptor srcNode;
private List<DatanodeDescriptor> containingNodes;
private List<DatanodeDescriptor> liveReplicaNodes;
private List<DatanodeStorageInfo> liveReplicaStorages;
private int additionalReplRequired;
private DatanodeDescriptor targets[];
private DatanodeStorageInfo targets[];
private int priority;
public ReplicationWork(Block block,
BlockCollection bc,
DatanodeDescriptor srcNode,
List<DatanodeDescriptor> containingNodes,
List<DatanodeDescriptor> liveReplicaNodes,
List<DatanodeStorageInfo> liveReplicaStorages,
int additionalReplRequired,
int priority) {
this.block = block;
this.bc = bc;
this.srcNode = srcNode;
this.containingNodes = containingNodes;
this.liveReplicaNodes = liveReplicaNodes;
this.liveReplicaStorages = liveReplicaStorages;
this.additionalReplRequired = additionalReplRequired;
this.priority = priority;
this.targets = null;
@ -3273,8 +3293,8 @@ public ReplicationWork(Block block,
private void chooseTargets(BlockPlacementPolicy blockplacement,
Set<Node> excludedNodes) {
targets = blockplacement.chooseTarget(bc.getName(),
additionalReplRequired, srcNode, liveReplicaNodes, false,
excludedNodes, block.getNumBytes());
additionalReplRequired, srcNode, liveReplicaStorages, false,
excludedNodes, block.getNumBytes(), StorageType.DEFAULT);
}
}

View File

@ -28,6 +28,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@ -67,13 +68,14 @@ public static class NotEnoughReplicasException extends Exception {
* @return array of DatanodeDescriptor instances chosen as target
* and sorted as a pipeline.
*/
public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
public abstract DatanodeStorageInfo[] chooseTarget(String srcPath,
int numOfReplicas,
Node writer,
List<DatanodeDescriptor> chosenNodes,
List<DatanodeStorageInfo> chosen,
boolean returnChosenNodes,
Set<Node> excludedNodes,
long blocksize);
long blocksize,
StorageType storageType);
/**
* Same as {@link #chooseTarget(String, int, Node, List, boolean,
@ -82,16 +84,19 @@ public abstract DatanodeDescriptor[] chooseTarget(String srcPath,
* is only a hint and due to cluster state, namenode may not be
* able to place the blocks on these datanodes.
*/
DatanodeDescriptor[] chooseTarget(String src,
DatanodeStorageInfo[] chooseTarget(String src,
int numOfReplicas, Node writer,
Set<Node> excludedNodes,
long blocksize, List<DatanodeDescriptor> favoredNodes) {
long blocksize,
List<DatanodeDescriptor> favoredNodes,
StorageType storageType) {
// This class does not provide the functionality of placing
// a block in favored datanodes. The implementations of this class
// are expected to provide this functionality
return chooseTarget(src, numOfReplicas, writer,
new ArrayList<DatanodeDescriptor>(numOfReplicas), false, excludedNodes,
blocksize);
new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
excludedNodes, blocksize, storageType);
}
/**

View File

@ -29,11 +29,14 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.net.NodeBase;
@ -103,99 +106,101 @@ public void initialize(Configuration conf, FSClusterStats stats,
}
@Override
public DatanodeDescriptor[] chooseTarget(String srcPath,
public DatanodeStorageInfo[] chooseTarget(String srcPath,
int numOfReplicas,
Node writer,
List<DatanodeDescriptor> chosenNodes,
List<DatanodeStorageInfo> chosenNodes,
boolean returnChosenNodes,
Set<Node> excludedNodes,
long blocksize) {
long blocksize,
StorageType storageType) {
return chooseTarget(numOfReplicas, writer, chosenNodes, returnChosenNodes,
excludedNodes, blocksize);
excludedNodes, blocksize, storageType);
}
@Override
DatanodeDescriptor[] chooseTarget(String src,
DatanodeStorageInfo[] chooseTarget(String src,
int numOfReplicas,
Node writer,
Set<Node> excludedNodes,
long blocksize,
List<DatanodeDescriptor> favoredNodes) {
List<DatanodeDescriptor> favoredNodes,
StorageType storageType) {
try {
if (favoredNodes == null || favoredNodes.size() == 0) {
// Favored nodes not specified, fall back to regular block placement.
return chooseTarget(src, numOfReplicas, writer,
new ArrayList<DatanodeDescriptor>(numOfReplicas), false,
excludedNodes, blocksize);
new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
excludedNodes, blocksize, storageType);
}
Set<Node> favoriteAndExcludedNodes = excludedNodes == null ?
new HashSet<Node>() : new HashSet<Node>(excludedNodes);
// Choose favored nodes
List<DatanodeDescriptor> results = new ArrayList<DatanodeDescriptor>();
List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>();
boolean avoidStaleNodes = stats != null
&& stats.isAvoidingStaleDataNodesForWrite();
for (int i = 0; i < Math.min(favoredNodes.size(), numOfReplicas); i++) {
DatanodeDescriptor favoredNode = favoredNodes.get(i);
// Choose a single node which is local to favoredNode.
// 'results' is updated within chooseLocalNode
DatanodeDescriptor target = chooseLocalNode(favoredNode,
final DatanodeStorageInfo target = chooseLocalStorage(favoredNode,
favoriteAndExcludedNodes, blocksize,
getMaxNodesPerRack(results,
numOfReplicas)[1], results, avoidStaleNodes);
getMaxNodesPerRack(results.size(), numOfReplicas)[1],
results, avoidStaleNodes, storageType);
if (target == null) {
LOG.warn("Could not find a target for file " + src
+ " with favored node " + favoredNode);
continue;
}
favoriteAndExcludedNodes.add(target);
favoriteAndExcludedNodes.add(target.getDatanodeDescriptor());
}
if (results.size() < numOfReplicas) {
// Not enough favored nodes, choose other nodes.
numOfReplicas -= results.size();
DatanodeDescriptor[] remainingTargets =
DatanodeStorageInfo[] remainingTargets =
chooseTarget(src, numOfReplicas, writer, results,
false, favoriteAndExcludedNodes, blocksize);
false, favoriteAndExcludedNodes, blocksize, storageType);
for (int i = 0; i < remainingTargets.length; i++) {
results.add(remainingTargets[i]);
}
}
return getPipeline(writer,
results.toArray(new DatanodeDescriptor[results.size()]));
results.toArray(new DatanodeStorageInfo[results.size()]));
} catch (NotEnoughReplicasException nr) {
// Fall back to regular block placement disregarding favored nodes hint
return chooseTarget(src, numOfReplicas, writer,
new ArrayList<DatanodeDescriptor>(numOfReplicas), false,
excludedNodes, blocksize);
new ArrayList<DatanodeStorageInfo>(numOfReplicas), false,
excludedNodes, blocksize, storageType);
}
}
/** This is the implementation. */
private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
private DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
Node writer,
List<DatanodeDescriptor> chosenNodes,
List<DatanodeStorageInfo> chosenStorage,
boolean returnChosenNodes,
Set<Node> excludedNodes,
long blocksize) {
long blocksize,
StorageType storageType) {
if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
return DatanodeDescriptor.EMPTY_ARRAY;
return DatanodeStorageInfo.EMPTY_ARRAY;
}
if (excludedNodes == null) {
excludedNodes = new HashSet<Node>();
}
int[] result = getMaxNodesPerRack(chosenNodes, numOfReplicas);
int[] result = getMaxNodesPerRack(chosenStorage.size(), numOfReplicas);
numOfReplicas = result[0];
int maxNodesPerRack = result[1];
List<DatanodeDescriptor> results =
new ArrayList<DatanodeDescriptor>(chosenNodes);
for (DatanodeDescriptor node:chosenNodes) {
final List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>(chosenStorage);
for (DatanodeStorageInfo storage : chosenStorage) {
// add localMachine and related nodes to excludedNodes
addToExcludedNodes(node, excludedNodes);
addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes);
}
if (!clusterMap.contains(writer)) {
@ -205,20 +210,19 @@ private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
boolean avoidStaleNodes = (stats != null
&& stats.isAvoidingStaleDataNodesForWrite());
Node localNode = chooseTarget(numOfReplicas, writer,
excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
if (!returnChosenNodes) {
results.removeAll(chosenNodes);
results.removeAll(chosenStorage);
}
// sorting nodes to form a pipeline
return getPipeline((writer==null)?localNode:writer,
results.toArray(new DatanodeDescriptor[results.size()]));
results.toArray(new DatanodeStorageInfo[results.size()]));
}
private int[] getMaxNodesPerRack(List<DatanodeDescriptor> chosenNodes,
int numOfReplicas) {
private int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) {
int clusterSize = clusterMap.getNumOfLeaves();
int totalNumOfReplicas = chosenNodes.size()+numOfReplicas;
int totalNumOfReplicas = numOfChosen + numOfReplicas;
if (totalNumOfReplicas > clusterSize) {
numOfReplicas -= (totalNumOfReplicas-clusterSize);
totalNumOfReplicas = clusterSize;
@ -243,8 +247,9 @@ private Node chooseTarget(int numOfReplicas,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
final boolean avoidStaleNodes) {
List<DatanodeStorageInfo> results,
final boolean avoidStaleNodes,
StorageType storageType) {
if (numOfReplicas == 0 || clusterMap.getNumOfLeaves()==0) {
return writer;
}
@ -253,7 +258,7 @@ private Node chooseTarget(int numOfReplicas,
int numOfResults = results.size();
boolean newBlock = (numOfResults==0);
if ((writer == null || !(writer instanceof DatanodeDescriptor)) && !newBlock) {
writer = results.get(0);
writer = results.get(0).getDatanodeDescriptor();
}
// Keep a copy of original excludedNodes
@ -261,42 +266,49 @@ private Node chooseTarget(int numOfReplicas,
new HashSet<Node>(excludedNodes) : null;
try {
if (numOfResults == 0) {
writer = chooseLocalNode(writer, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
writer = chooseLocalStorage(writer, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType)
.getDatanodeDescriptor();
if (--numOfReplicas == 0) {
return writer;
}
}
final DatanodeDescriptor dn0 = results.get(0).getDatanodeDescriptor();
if (numOfResults <= 1) {
chooseRemoteRack(1, results.get(0), excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType);
if (--numOfReplicas == 0) {
return writer;
}
}
if (numOfResults <= 2) {
if (clusterMap.isOnSameRack(results.get(0), results.get(1))) {
chooseRemoteRack(1, results.get(0), excludedNodes,
blocksize, maxNodesPerRack,
results, avoidStaleNodes);
final DatanodeDescriptor dn1 = results.get(1).getDatanodeDescriptor();
if (clusterMap.isOnSameRack(dn0, dn1)) {
chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType);
} else if (newBlock){
chooseLocalRack(results.get(1), excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
chooseLocalRack(dn1, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType);
} else {
chooseLocalRack(writer, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes);
results, avoidStaleNodes, storageType);
}
if (--numOfReplicas == 0) {
return writer;
}
}
chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
maxNodesPerRack, results, avoidStaleNodes, storageType);
} catch (NotEnoughReplicasException e) {
LOG.warn("Not able to place enough replicas, still in need of "
+ (totalReplicasExpected - results.size()) + " to reach "
+ totalReplicasExpected + "\n"
+ e.getMessage());
final String message = "Failed to place enough replicas, still in need of "
+ (totalReplicasExpected - results.size()) + " to reach "
+ totalReplicasExpected + ".";
if (LOG.isTraceEnabled()) {
LOG.trace(message, e);
} else {
LOG.warn(message + " " + e.getMessage());
}
if (avoidStaleNodes) {
// Retry chooseTarget again, this time not avoiding stale nodes.
@ -304,14 +316,14 @@ private Node chooseTarget(int numOfReplicas,
// not chosen because they were stale, decommissioned, etc.
// We need to additionally exclude the nodes that were added to the
// result list in the successful calls to choose*() above.
for (Node node : results) {
oldExcludedNodes.add(node);
for (DatanodeStorageInfo resultStorage : results) {
oldExcludedNodes.add(resultStorage.getDatanodeDescriptor());
}
// Set numOfReplicas, since it can get out of sync with the result list
// if the NotEnoughReplicasException was thrown in chooseRandom().
numOfReplicas = totalReplicasExpected - results.size();
return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
maxNodesPerRack, results, false);
maxNodesPerRack, results, false, storageType);
}
}
return writer;
@ -321,32 +333,36 @@ private Node chooseTarget(int numOfReplicas,
* Choose <i>localMachine</i> as the target.
* if <i>localMachine</i> is not available,
* choose a node on the same rack
* @return the chosen node
* @return the chosen storage
*/
protected DatanodeDescriptor chooseLocalNode(Node localMachine,
protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
boolean avoidStaleNodes)
List<DatanodeStorageInfo> results,
boolean avoidStaleNodes,
StorageType storageType)
throws NotEnoughReplicasException {
// if no local machine, randomly choose one node
if (localMachine == null)
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
maxNodesPerRack, results, avoidStaleNodes, storageType);
if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
// otherwise try local machine first
if (excludedNodes.add(localMachine)) { // was not in the excluded list
if (addIfIsGoodTarget(localDatanode, excludedNodes, blocksize,
maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
return localDatanode;
for(DatanodeStorageInfo localStorage : DFSUtil.shuffle(
localDatanode.getStorageInfos())) {
if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) {
return localStorage;
}
}
}
}
// try a node on local rack
return chooseLocalRack(localMachine, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
/**
@ -368,27 +384,29 @@ protected int addToExcludedNodes(DatanodeDescriptor localMachine,
* in the cluster.
* @return the chosen node
*/
protected DatanodeDescriptor chooseLocalRack(Node localMachine,
protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
boolean avoidStaleNodes)
List<DatanodeStorageInfo> results,
boolean avoidStaleNodes,
StorageType storageType)
throws NotEnoughReplicasException {
// no local machine, so choose a random machine
if (localMachine == null) {
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
// choose one from the local rack
try {
return chooseRandom(localMachine.getNetworkLocation(), excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
} catch (NotEnoughReplicasException e1) {
// find the second replica
DatanodeDescriptor newLocal=null;
for(DatanodeDescriptor nextNode : results) {
for(DatanodeStorageInfo resultStorage : results) {
DatanodeDescriptor nextNode = resultStorage.getDatanodeDescriptor();
if (nextNode != localMachine) {
newLocal = nextNode;
break;
@ -397,16 +415,16 @@ protected DatanodeDescriptor chooseLocalRack(Node localMachine,
if (newLocal != null) {
try {
return chooseRandom(newLocal.getNetworkLocation(), excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
} catch(NotEnoughReplicasException e2) {
//otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
} else {
//otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
}
}
@ -423,48 +441,51 @@ protected void chooseRemoteRack(int numOfReplicas,
Set<Node> excludedNodes,
long blocksize,
int maxReplicasPerRack,
List<DatanodeDescriptor> results,
boolean avoidStaleNodes)
List<DatanodeStorageInfo> results,
boolean avoidStaleNodes,
StorageType storageType)
throws NotEnoughReplicasException {
int oldNumOfReplicas = results.size();
// randomly choose one node from remote racks
try {
chooseRandom(numOfReplicas, "~" + localMachine.getNetworkLocation(),
excludedNodes, blocksize, maxReplicasPerRack, results,
avoidStaleNodes);
avoidStaleNodes, storageType);
} catch (NotEnoughReplicasException e) {
chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
localMachine.getNetworkLocation(), excludedNodes, blocksize,
maxReplicasPerRack, results, avoidStaleNodes);
maxReplicasPerRack, results, avoidStaleNodes, storageType);
}
}
/**
* Randomly choose one target from the given <i>scope</i>.
* @return the chosen node, if there is any.
* @return the chosen storage, if there is any.
*/
protected DatanodeDescriptor chooseRandom(String scope,
protected DatanodeStorageInfo chooseRandom(String scope,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
boolean avoidStaleNodes)
List<DatanodeStorageInfo> results,
boolean avoidStaleNodes,
StorageType storageType)
throws NotEnoughReplicasException {
return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes);
results, avoidStaleNodes, storageType);
}
/**
* Randomly choose <i>numOfReplicas</i> targets from the given <i>scope</i>.
* @return the first chosen node, if there is any.
*/
protected DatanodeDescriptor chooseRandom(int numOfReplicas,
protected DatanodeStorageInfo chooseRandom(int numOfReplicas,
String scope,
Set<Node> excludedNodes,
long blocksize,
int maxNodesPerRack,
List<DatanodeDescriptor> results,
boolean avoidStaleNodes)
List<DatanodeStorageInfo> results,
boolean avoidStaleNodes,
StorageType storageType)
throws NotEnoughReplicasException {
int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
@ -476,24 +497,32 @@ protected DatanodeDescriptor chooseRandom(int numOfReplicas,
builder.append("[");
}
boolean badTarget = false;
DatanodeDescriptor firstChosen = null;
DatanodeStorageInfo firstChosen = null;
while(numOfReplicas > 0 && numOfAvailableNodes > 0) {
DatanodeDescriptor chosenNode =
(DatanodeDescriptor)clusterMap.chooseRandom(scope);
if (excludedNodes.add(chosenNode)) { //was not in the excluded list
numOfAvailableNodes--;
int newExcludedNodes = addIfIsGoodTarget(chosenNode, excludedNodes,
blocksize, maxNodesPerRack, considerLoad, results, avoidStaleNodes);
if (newExcludedNodes >= 0) {
numOfReplicas--;
if (firstChosen == null) {
firstChosen = chosenNode;
final DatanodeStorageInfo[] storages = DFSUtil.shuffle(
chosenNode.getStorageInfos());
int i;
for(i = 0; i < storages.length; i++) {
final int newExcludedNodes = addIfIsGoodTarget(storages[i],
excludedNodes, blocksize, maxNodesPerRack, considerLoad, results,
avoidStaleNodes, storageType);
if (newExcludedNodes >= 0) {
numOfReplicas--;
if (firstChosen == null) {
firstChosen = storages[i];
}
numOfAvailableNodes -= newExcludedNodes;
break;
}
numOfAvailableNodes -= newExcludedNodes;
} else {
badTarget = true;
}
// If no candidate storage was found on this DN then set badTarget.
badTarget = (i == storages.length);
}
}
@ -512,43 +541,46 @@ protected DatanodeDescriptor chooseRandom(int numOfReplicas,
}
/**
* If the given node is a good target, add it to the result list and
* If the given storage is a good target, add it to the result list and
* update the set of excluded nodes.
* @return -1 if the given is not a good target;
* otherwise, return the number of nodes added to excludedNodes set.
*/
int addIfIsGoodTarget(DatanodeDescriptor node,
int addIfIsGoodTarget(DatanodeStorageInfo storage,
Set<Node> excludedNodes,
long blockSize,
int maxNodesPerRack,
boolean considerLoad,
List<DatanodeDescriptor> results,
boolean avoidStaleNodes) {
if (isGoodTarget(node, blockSize, maxNodesPerRack, considerLoad,
results, avoidStaleNodes)) {
results.add(node);
List<DatanodeStorageInfo> results,
boolean avoidStaleNodes,
StorageType storageType) {
if (isGoodTarget(storage, blockSize, maxNodesPerRack, considerLoad,
results, avoidStaleNodes, storageType)) {
results.add(storage);
// add node and related nodes to excludedNode
return addToExcludedNodes(node, excludedNodes);
return addToExcludedNodes(storage.getDatanodeDescriptor(), excludedNodes);
} else {
return -1;
}
}
private static void logNodeIsNotChosen(DatanodeDescriptor node, String reason) {
private static void logNodeIsNotChosen(DatanodeStorageInfo storage, String reason) {
if (LOG.isDebugEnabled()) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
// build the error message for later use.
debugLoggingBuilder.get()
.append(node).append(": ")
.append("Node ").append(NodeBase.getPath(node))
.append("Storage ").append(storage)
.append("at node ").append(NodeBase.getPath(node))
.append(" is not chosen because ")
.append(reason);
}
}
/**
* Determine if a node is a good target.
* Determine if a storage is a good target.
*
* @param node The target node
* @param storage The target storage
* @param blockSize Size of block
* @param maxTargetPerRack Maximum number of targets per rack. The value of
* this parameter depends on the number of racks in
@ -561,32 +593,43 @@ private static void logNodeIsNotChosen(DatanodeDescriptor node, String reason) {
* does not have too much load,
* and the rack does not have too many nodes.
*/
private boolean isGoodTarget(DatanodeDescriptor node,
private boolean isGoodTarget(DatanodeStorageInfo storage,
long blockSize, int maxTargetPerRack,
boolean considerLoad,
List<DatanodeDescriptor> results,
boolean avoidStaleNodes) {
// check if the node is (being) decommissed
List<DatanodeStorageInfo> results,
boolean avoidStaleNodes,
StorageType storageType) {
if (storage.getStorageType() != storageType) {
logNodeIsNotChosen(storage,
"storage types do not match, where the expected storage type is "
+ storageType);
return false;
}
if (storage.getState() == State.READ_ONLY) {
logNodeIsNotChosen(storage, "storage is read-only");
return false;
}
DatanodeDescriptor node = storage.getDatanodeDescriptor();
// check if the node is (being) decommissioned
if (node.isDecommissionInProgress() || node.isDecommissioned()) {
logNodeIsNotChosen(node, "the node is (being) decommissioned ");
logNodeIsNotChosen(storage, "the node is (being) decommissioned ");
return false;
}
if (avoidStaleNodes) {
if (node.isStale(this.staleInterval)) {
logNodeIsNotChosen(node, "the node is stale ");
logNodeIsNotChosen(storage, "the node is stale ");
return false;
}
}
long remaining = node.getRemaining() -
(node.getBlocksScheduled() * blockSize);
// check the remaining capacity of the target machine
if (blockSize* HdfsConstants.MIN_BLOCKS_FOR_WRITE>remaining) {
logNodeIsNotChosen(node, "the node does not have enough space ");
final long requiredSize = blockSize * HdfsConstants.MIN_BLOCKS_FOR_WRITE;
final long scheduledSize = blockSize * node.getBlocksScheduled();
if (requiredSize > node.getRemaining() - scheduledSize) {
logNodeIsNotChosen(storage, "the node does not have enough space ");
return false;
}
// check the communication traffic of the target machine
if (considerLoad) {
double avgLoad = 0;
@ -595,7 +638,7 @@ private boolean isGoodTarget(DatanodeDescriptor node,
avgLoad = (double)stats.getTotalLoad()/size;
}
if (node.getXceiverCount() > (2.0 * avgLoad)) {
logNodeIsNotChosen(node, "the node is too busy ");
logNodeIsNotChosen(storage, "the node is too busy ");
return false;
}
}
@ -603,13 +646,14 @@ private boolean isGoodTarget(DatanodeDescriptor node,
// check if the target rack has chosen too many nodes
String rackname = node.getNetworkLocation();
int counter=1;
for(Node result : results) {
if (rackname.equals(result.getNetworkLocation())) {
for(DatanodeStorageInfo resultStorage : results) {
if (rackname.equals(
resultStorage.getDatanodeDescriptor().getNetworkLocation())) {
counter++;
}
}
if (counter>maxTargetPerRack) {
logNodeIsNotChosen(node, "the rack has too many chosen nodes ");
logNodeIsNotChosen(storage, "the rack has too many chosen nodes ");
return false;
}
return true;
@ -621,37 +665,40 @@ private boolean isGoodTarget(DatanodeDescriptor node,
* starts from the writer and traverses all <i>nodes</i>
* This is basically a traveling salesman problem.
*/
private DatanodeDescriptor[] getPipeline(Node writer,
DatanodeDescriptor[] nodes) {
if (nodes.length==0) return nodes;
private DatanodeStorageInfo[] getPipeline(Node writer,
DatanodeStorageInfo[] storages) {
if (storages.length == 0) {
return storages;
}
synchronized(clusterMap) {
int index=0;
if (writer == null || !clusterMap.contains(writer)) {
writer = nodes[0];
writer = storages[0].getDatanodeDescriptor();
}
for(;index<nodes.length; index++) {
DatanodeDescriptor shortestNode = nodes[index];
int shortestDistance = clusterMap.getDistance(writer, shortestNode);
for(; index < storages.length; index++) {
DatanodeStorageInfo shortestStorage = storages[index];
int shortestDistance = clusterMap.getDistance(writer,
shortestStorage.getDatanodeDescriptor());
int shortestIndex = index;
for(int i=index+1; i<nodes.length; i++) {
DatanodeDescriptor currentNode = nodes[i];
int currentDistance = clusterMap.getDistance(writer, currentNode);
for(int i = index + 1; i < storages.length; i++) {
int currentDistance = clusterMap.getDistance(writer,
storages[i].getDatanodeDescriptor());
if (shortestDistance>currentDistance) {
shortestDistance = currentDistance;
shortestNode = currentNode;
shortestStorage = storages[i];
shortestIndex = i;
}
}
//switch position index & shortestIndex
if (index != shortestIndex) {
nodes[shortestIndex] = nodes[index];
nodes[index] = shortestNode;
storages[shortestIndex] = storages[index];
storages[index] = shortestStorage;
}
writer = shortestNode;
writer = shortestStorage.getDatanodeDescriptor();
}
}
return nodes;
return storages;
}
@Override

View File

@ -25,6 +25,8 @@
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
import org.apache.hadoop.net.NetworkTopology;
@ -63,83 +65,87 @@ public void initialize(Configuration conf, FSClusterStats stats,
* @return the chosen node
*/
@Override
protected DatanodeDescriptor chooseLocalNode(Node localMachine,
protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
List<DatanodeDescriptor> results, boolean avoidStaleNodes)
throws NotEnoughReplicasException {
List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
StorageType storageType) throws NotEnoughReplicasException {
// if no local machine, randomly choose one node
if (localMachine == null)
return chooseRandom(NodeBase.ROOT, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
// otherwise try local machine first
if (localMachine instanceof DatanodeDescriptor) {
DatanodeDescriptor localDataNode = (DatanodeDescriptor)localMachine;
// otherwise try local machine first
if (excludedNodes.add(localMachine)) { // was not in the excluded list
if (addIfIsGoodTarget(localDataNode, excludedNodes, blocksize,
maxNodesPerRack, false, results, avoidStaleNodes) >= 0) {
return localDataNode;
for(DatanodeStorageInfo localStorage : DFSUtil.shuffle(
localDataNode.getStorageInfos())) {
if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) {
return localStorage;
}
}
}
}
// try a node on local node group
DatanodeDescriptor chosenNode = chooseLocalNodeGroup(
DatanodeStorageInfo chosenStorage = chooseLocalNodeGroup(
(NetworkTopologyWithNodeGroup)clusterMap, localMachine, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
if (chosenNode != null) {
return chosenNode;
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
if (chosenStorage != null) {
return chosenStorage;
}
// try a node on local rack
return chooseLocalRack(localMachine, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
}
/** @return the node of the second replica */
private static DatanodeDescriptor secondNode(Node localMachine,
List<DatanodeStorageInfo> results) {
// find the second replica
for(DatanodeStorageInfo nextStorage : results) {
DatanodeDescriptor nextNode = nextStorage.getDatanodeDescriptor();
if (nextNode != localMachine) {
return nextNode;
}
}
return null;
}
/**
* {@inheritDoc}
*/
@Override
protected DatanodeDescriptor chooseLocalRack(Node localMachine,
protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
List<DatanodeDescriptor> results, boolean avoidStaleNodes)
throws NotEnoughReplicasException {
List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
StorageType storageType) throws NotEnoughReplicasException {
// no local machine, so choose a random machine
if (localMachine == null) {
return chooseRandom(NodeBase.ROOT, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
// choose one from the local rack, but off-nodegroup
try {
return chooseRandom(NetworkTopology.getFirstHalf(
localMachine.getNetworkLocation()),
excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes);
final String scope = NetworkTopology.getFirstHalf(localMachine.getNetworkLocation());
return chooseRandom(scope, excludedNodes, blocksize, maxNodesPerRack,
results, avoidStaleNodes, storageType);
} catch (NotEnoughReplicasException e1) {
// find the second replica
DatanodeDescriptor newLocal=null;
for(DatanodeDescriptor nextNode : results) {
if (nextNode != localMachine) {
newLocal = nextNode;
break;
}
}
final DatanodeDescriptor newLocal = secondNode(localMachine, results);
if (newLocal != null) {
try {
return chooseRandom(clusterMap.getRack(newLocal.getNetworkLocation()),
excludedNodes, blocksize, maxNodesPerRack, results,
avoidStaleNodes);
return chooseRandom(
clusterMap.getRack(newLocal.getNetworkLocation()), excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
} catch(NotEnoughReplicasException e2) {
//otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes,
blocksize, maxNodesPerRack, results,
avoidStaleNodes);
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
} else {
//otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes,
blocksize, maxNodesPerRack, results,
avoidStaleNodes);
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
}
}
@ -150,8 +156,9 @@ protected DatanodeDescriptor chooseLocalRack(Node localMachine,
@Override
protected void chooseRemoteRack(int numOfReplicas,
DatanodeDescriptor localMachine, Set<Node> excludedNodes,
long blocksize, int maxReplicasPerRack, List<DatanodeDescriptor> results,
boolean avoidStaleNodes) throws NotEnoughReplicasException {
long blocksize, int maxReplicasPerRack, List<DatanodeStorageInfo> results,
boolean avoidStaleNodes, StorageType storageType)
throws NotEnoughReplicasException {
int oldNumOfReplicas = results.size();
final String rackLocation = NetworkTopology.getFirstHalf(
@ -159,12 +166,12 @@ protected void chooseRemoteRack(int numOfReplicas,
try {
// randomly choose from remote racks
chooseRandom(numOfReplicas, "~" + rackLocation, excludedNodes, blocksize,
maxReplicasPerRack, results, avoidStaleNodes);
maxReplicasPerRack, results, avoidStaleNodes, storageType);
} catch (NotEnoughReplicasException e) {
// fall back to the local rack
chooseRandom(numOfReplicas - (results.size() - oldNumOfReplicas),
rackLocation, excludedNodes, blocksize,
maxReplicasPerRack, results, avoidStaleNodes);
maxReplicasPerRack, results, avoidStaleNodes, storageType);
}
}
@ -174,43 +181,40 @@ protected void chooseRemoteRack(int numOfReplicas,
* if still no such node is available, choose a random node in the cluster.
* @return the chosen node
*/
private DatanodeDescriptor chooseLocalNodeGroup(
private DatanodeStorageInfo chooseLocalNodeGroup(
NetworkTopologyWithNodeGroup clusterMap, Node localMachine,
Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
List<DatanodeDescriptor> results, boolean avoidStaleNodes)
throws NotEnoughReplicasException {
List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
StorageType storageType) throws NotEnoughReplicasException {
// no local machine, so choose a random machine
if (localMachine == null) {
return chooseRandom(NodeBase.ROOT, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
// choose one from the local node group
try {
return chooseRandom(clusterMap.getNodeGroup(localMachine.getNetworkLocation()),
excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
return chooseRandom(
clusterMap.getNodeGroup(localMachine.getNetworkLocation()),
excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes,
storageType);
} catch (NotEnoughReplicasException e1) {
// find the second replica
DatanodeDescriptor newLocal=null;
for(DatanodeDescriptor nextNode : results) {
if (nextNode != localMachine) {
newLocal = nextNode;
break;
}
}
final DatanodeDescriptor newLocal = secondNode(localMachine, results);
if (newLocal != null) {
try {
return chooseRandom(clusterMap.getNodeGroup(newLocal.getNetworkLocation()),
excludedNodes, blocksize, maxNodesPerRack, results, avoidStaleNodes);
return chooseRandom(
clusterMap.getNodeGroup(newLocal.getNetworkLocation()),
excludedNodes, blocksize, maxNodesPerRack, results,
avoidStaleNodes, storageType);
} catch(NotEnoughReplicasException e2) {
//otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
} else {
//otherwise randomly choose one from the network
return chooseRandom(NodeBase.ROOT, excludedNodes,
blocksize, maxNodesPerRack, results, avoidStaleNodes);
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
maxNodesPerRack, results, avoidStaleNodes, storageType);
}
}
}

View File

@ -29,11 +29,11 @@
* the datanodes that store the block.
*/
class BlocksMap {
private static class NodeIterator implements Iterator<DatanodeDescriptor> {
private static class StorageIterator implements Iterator<DatanodeStorageInfo> {
private BlockInfo blockInfo;
private int nextIdx = 0;
NodeIterator(BlockInfo blkInfo) {
StorageIterator(BlockInfo blkInfo) {
this.blockInfo = blkInfo;
}
@ -44,8 +44,8 @@ public boolean hasNext() {
}
@Override
public DatanodeDescriptor next() {
return blockInfo.getDatanode(nextIdx++);
public DatanodeStorageInfo next() {
return blockInfo.getStorageInfo(nextIdx++);
}
@Override
@ -115,18 +115,23 @@ BlockInfo getStoredBlock(Block b) {
/**
* Searches for the block in the BlocksMap and
* returns Iterator that iterates through the nodes the block belongs to.
* returns {@link Iterable} of the storages the block belongs to.
*/
Iterator<DatanodeDescriptor> nodeIterator(Block b) {
return nodeIterator(blocks.get(b));
Iterable<DatanodeStorageInfo> getStorages(Block b) {
return getStorages(blocks.get(b));
}
/**
* For a block that has already been retrieved from the BlocksMap
* returns Iterator that iterates through the nodes the block belongs to.
* returns {@link Iterable} of the storages the block belongs to.
*/
Iterator<DatanodeDescriptor> nodeIterator(BlockInfo storedBlock) {
return new NodeIterator(storedBlock);
Iterable<DatanodeStorageInfo> getStorages(final BlockInfo storedBlock) {
return new Iterable<DatanodeStorageInfo>() {
@Override
public Iterator<DatanodeStorageInfo> iterator() {
return new StorageIterator(storedBlock);
}
};
}
/** counts number of containing nodes. Better than using iterator. */

View File

@ -17,20 +17,17 @@
*/
package org.apache.hadoop.hdfs.server.blockmanagement;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
import java.util.Set;
import java.util.TreeSet;
import java.util.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
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.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.hdfs.util.LightWeightHashSet;
import org.apache.hadoop.util.Time;
@ -42,6 +39,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class DatanodeDescriptor extends DatanodeInfo {
public static final Log LOG = LogFactory.getLog(DatanodeDescriptor.class);
public static final DatanodeDescriptor[] EMPTY_ARRAY = {};
// Stores status of decommissioning.
@ -53,9 +51,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
@InterfaceStability.Evolving
public static class BlockTargetPair {
public final Block block;
public final DatanodeDescriptor[] targets;
public final DatanodeStorageInfo[] targets;
BlockTargetPair(Block block, DatanodeDescriptor[] targets) {
BlockTargetPair(Block block, DatanodeStorageInfo[] targets) {
this.block = block;
this.targets = targets;
}
@ -98,30 +96,14 @@ synchronized void clear() {
}
}
private volatile BlockInfo blockList = null;
private int numBlocks = 0;
private final Map<String, DatanodeStorageInfo> storageMap =
new HashMap<String, DatanodeStorageInfo>();
// isAlive == heartbeats.contains(this)
// This is an optimization, because contains takes O(n) time on Arraylist
public boolean isAlive = false;
public boolean needKeyUpdate = false;
/**
* Set to false on any NN failover, and reset to true
* whenever a block report is received.
*/
private boolean heartbeatedSinceFailover = false;
/**
* At startup or at any failover, the DNs in the cluster may
* have pending block deletions from a previous incarnation
* of the NameNode. Thus, we consider their block contents
* stale until we have received a block report. When a DN
* is considered stale, any replicas on it are transitively
* considered stale. If any block has at least one stale replica,
* then no invalidations will be processed for this block.
* See HDFS-1972.
*/
private boolean blockContentsStale = true;
// A system administrator can tune the balancer bandwidth parameter
// (dfs.balance.bandwidthPerSec) dynamically by calling
@ -140,7 +122,7 @@ synchronized void clear() {
private LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
/* Variables for maintaining number of blocks scheduled to be written to
* this datanode. This count is approximate and might be slightly bigger
* this storage. This count is approximate and might be slightly bigger
* in case of errors (e.g. datanode does not report if an error occurs
* while writing the block).
*/
@ -150,9 +132,6 @@ synchronized void clear() {
private static final int BLOCKS_SCHEDULED_ROLL_INTERVAL = 600*1000; //10min
private int volumeFailures = 0;
/** Set to false after processing first block report */
private boolean firstBlockReport = true;
/**
* When set to true, the node is not in include list and is not allowed
* to communicate with the namenode
@ -164,7 +143,8 @@ synchronized void clear() {
* @param nodeID id of the data node
*/
public DatanodeDescriptor(DatanodeID nodeID) {
this(nodeID, 0L, 0L, 0L, 0L, 0, 0);
super(nodeID);
updateHeartbeat(StorageReport.EMPTY_ARRAY, 0, 0);
}
/**
@ -174,95 +154,60 @@ public DatanodeDescriptor(DatanodeID nodeID) {
*/
public DatanodeDescriptor(DatanodeID nodeID,
String networkLocation) {
this(nodeID, networkLocation, 0L, 0L, 0L, 0L, 0, 0);
}
/**
* DatanodeDescriptor constructor
* @param nodeID id of the data node
* @param capacity capacity of the data node
* @param dfsUsed space used by the data node
* @param remaining remaining capacity of the data node
* @param bpused space used by the block pool corresponding to this namenode
* @param xceiverCount # of data transfers at the data node
*/
public DatanodeDescriptor(DatanodeID nodeID,
long capacity,
long dfsUsed,
long remaining,
long bpused,
int xceiverCount,
int failedVolumes) {
super(nodeID);
updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount,
failedVolumes);
}
/**
* DatanodeDescriptor constructor
* @param nodeID id of the data node
* @param networkLocation location of the data node in network
* @param capacity capacity of the data node, including space used by non-dfs
* @param dfsUsed the used space by dfs datanode
* @param remaining remaining capacity of the data node
* @param bpused space used by the block pool corresponding to this namenode
* @param xceiverCount # of data transfers at the data node
*/
public DatanodeDescriptor(DatanodeID nodeID,
String networkLocation,
long capacity,
long dfsUsed,
long remaining,
long bpused,
int xceiverCount,
int failedVolumes) {
super(nodeID, networkLocation);
updateHeartbeat(capacity, dfsUsed, remaining, bpused, xceiverCount,
failedVolumes);
updateHeartbeat(StorageReport.EMPTY_ARRAY, 0, 0);
}
/**
* Add datanode to the block.
* Add block to the head of the list of blocks belonging to the data-node.
* Add data-node to the block. Add block to the head of the list of blocks
* belonging to the data-node.
*/
public boolean addBlock(BlockInfo b) {
if(!b.addNode(this))
return false;
// add to the head of the data-node list
blockList = b.listInsert(blockList, this);
numBlocks++;
return true;
public boolean addBlock(String storageID, BlockInfo b) {
DatanodeStorageInfo s = getStorageInfo(storageID);
if (s != null) {
return s.addBlock(b);
}
return false;
}
/**
* Remove block from the list of blocks belonging to the data-node.
* Remove datanode from the block.
*/
public boolean removeBlock(BlockInfo b) {
blockList = b.listRemove(blockList, this);
if ( b.removeNode(this) ) {
numBlocks--;
return true;
} else {
return false;
DatanodeStorageInfo getStorageInfo(String storageID) {
synchronized (storageMap) {
return storageMap.get(storageID);
}
}
DatanodeStorageInfo[] getStorageInfos() {
synchronized (storageMap) {
final Collection<DatanodeStorageInfo> storages = storageMap.values();
return storages.toArray(new DatanodeStorageInfo[storages.size()]);
}
}
/**
* Move block to the head of the list of blocks belonging to the data-node.
* @return the index of the head of the blockList
* Remove block from the list of blocks belonging to the data-node. Remove
* data-node from the block.
*/
int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
return curIndex;
boolean removeBlock(BlockInfo b) {
int index = b.findStorageInfo(this);
// if block exists on this datanode
if (index >= 0) {
DatanodeStorageInfo s = b.getStorageInfo(index);
if (s != null) {
return s.removeBlock(b);
}
}
return false;
}
/**
* Used for testing only
* @return the head of the blockList
* Remove block from the list of blocks belonging to the data-node. Remove
* data-node from the block.
*/
protected BlockInfo getHead(){
return blockList;
boolean removeBlock(String storageID, BlockInfo b) {
DatanodeStorageInfo s = getStorageInfo(storageID);
if (s != null) {
return s.removeBlock(b);
}
return false;
}
/**
@ -273,9 +218,12 @@ protected BlockInfo getHead(){
* @return the new block
*/
public BlockInfo replaceBlock(BlockInfo oldBlock, BlockInfo newBlock) {
boolean done = removeBlock(oldBlock);
int index = oldBlock.findStorageInfo(this);
DatanodeStorageInfo s = oldBlock.getStorageInfo(index);
boolean done = s.removeBlock(oldBlock);
assert done : "Old block should belong to the data-node when replacing";
done = addBlock(newBlock);
done = s.addBlock(newBlock);
assert done : "New block should not belong to the data-node when replacing";
return newBlock;
}
@ -286,7 +234,6 @@ public void resetBlocks() {
setBlockPoolUsed(0);
setDfsUsed(0);
setXceiverCount(0);
this.blockList = null;
this.invalidateBlocks.clear();
this.volumeFailures = 0;
}
@ -300,63 +247,95 @@ public void clearBlockQueues() {
}
public int numBlocks() {
return numBlocks;
int blocks = 0;
for (DatanodeStorageInfo entry : getStorageInfos()) {
blocks += entry.numBlocks();
}
return blocks;
}
/**
* Updates stats from datanode heartbeat.
*/
public void updateHeartbeat(long capacity, long dfsUsed, long remaining,
long blockPoolUsed, int xceiverCount, int volFailures) {
setCapacity(capacity);
setRemaining(remaining);
setBlockPoolUsed(blockPoolUsed);
setDfsUsed(dfsUsed);
public void updateHeartbeat(StorageReport[] reports,
int xceiverCount, int volFailures) {
long totalCapacity = 0;
long totalRemaining = 0;
long totalBlockPoolUsed = 0;
long totalDfsUsed = 0;
setXceiverCount(xceiverCount);
setLastUpdate(Time.now());
this.volumeFailures = volFailures;
this.heartbeatedSinceFailover = true;
for (StorageReport report : reports) {
DatanodeStorageInfo storage = storageMap.get(report.getStorageID());
if (storage == null) {
// This is seen during cluster initialization when the heartbeat
// is received before the initial block reports from each storage.
storage = updateStorage(new DatanodeStorage(report.getStorageID()));
}
storage.receivedHeartbeat(report);
totalCapacity += report.getCapacity();
totalRemaining += report.getRemaining();
totalBlockPoolUsed += report.getBlockPoolUsed();
totalDfsUsed += report.getDfsUsed();
}
rollBlocksScheduled(getLastUpdate());
// Update total metrics for the node.
setCapacity(totalCapacity);
setRemaining(totalRemaining);
setBlockPoolUsed(totalBlockPoolUsed);
setDfsUsed(totalDfsUsed);
}
/**
* Iterates over the list of blocks belonging to the datanode.
*/
public static class BlockIterator implements Iterator<BlockInfo> {
private BlockInfo current;
private DatanodeDescriptor node;
BlockIterator(BlockInfo head, DatanodeDescriptor dn) {
this.current = head;
this.node = dn;
private static class BlockIterator implements Iterator<BlockInfo> {
private int index = 0;
private final List<Iterator<BlockInfo>> iterators;
private BlockIterator(final DatanodeStorageInfo... storages) {
List<Iterator<BlockInfo>> iterators = new ArrayList<Iterator<BlockInfo>>();
for (DatanodeStorageInfo e : storages) {
iterators.add(e.getBlockIterator());
}
this.iterators = Collections.unmodifiableList(iterators);
}
@Override
public boolean hasNext() {
return current != null;
update();
return !iterators.isEmpty() && iterators.get(index).hasNext();
}
@Override
public BlockInfo next() {
BlockInfo res = current;
current = current.getNext(current.findDatanode(node));
return res;
update();
return iterators.get(index).next();
}
@Override
public void remove() {
throw new UnsupportedOperationException("Sorry. can't remove.");
public void remove() {
throw new UnsupportedOperationException("Remove unsupported.");
}
private void update() {
while(index < iterators.size() - 1 && !iterators.get(index).hasNext()) {
index++;
}
}
}
public Iterator<BlockInfo> getBlockIterator() {
return new BlockIterator(this.blockList, this);
Iterator<BlockInfo> getBlockIterator() {
return new BlockIterator(getStorageInfos());
}
Iterator<BlockInfo> getBlockIterator(final String storageID) {
return new BlockIterator(getStorageInfo(storageID));
}
/**
* Store block replication work.
*/
void addBlockToBeReplicated(Block block, DatanodeDescriptor[] targets) {
void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) {
assert(block != null && targets != null && targets.length > 0);
replicateBlocks.offer(new BlockTargetPair(block, targets));
}
@ -431,18 +410,14 @@ public Block[] getInvalidateBlocks(int maxblocks) {
public int getBlocksScheduled() {
return currApproxBlocksScheduled + prevApproxBlocksScheduled;
}
/**
* Increments counter for number of blocks scheduled.
*/
public void incBlocksScheduled() {
/** Increment the number of blocks scheduled. */
void incrementBlocksScheduled() {
currApproxBlocksScheduled++;
}
/**
* Decrements counter for number of blocks scheduled.
*/
void decBlocksScheduled() {
/** Decrement the number of blocks scheduled. */
void decrementBlocksScheduled() {
if (prevApproxBlocksScheduled > 0) {
prevApproxBlocksScheduled--;
} else if (currApproxBlocksScheduled > 0) {
@ -451,12 +426,9 @@ void decBlocksScheduled() {
// its ok if both counters are zero.
}
/**
* Adjusts curr and prev number of blocks scheduled every few minutes.
*/
/** Adjusts curr and prev number of blocks scheduled every few minutes. */
private void rollBlocksScheduled(long now) {
if ((now - lastBlocksScheduledRollTime) >
BLOCKS_SCHEDULED_ROLL_INTERVAL) {
if (now - lastBlocksScheduledRollTime > BLOCKS_SCHEDULED_ROLL_INTERVAL) {
prevApproxBlocksScheduled = currApproxBlocksScheduled;
currApproxBlocksScheduled = 0;
lastBlocksScheduledRollTime = now;
@ -552,7 +524,11 @@ public int getVolumeFailures() {
@Override
public void updateRegInfo(DatanodeID nodeReg) {
super.updateRegInfo(nodeReg);
firstBlockReport = true; // must re-process IBR after re-registration
// must re-process IBR after re-registration
for(DatanodeStorageInfo storage : getStorageInfos()) {
storage.setBlockReportCount(0);
}
}
/**
@ -569,26 +545,6 @@ public void setBalancerBandwidth(long bandwidth) {
this.bandwidth = bandwidth;
}
public boolean areBlockContentsStale() {
return blockContentsStale;
}
public void markStaleAfterFailover() {
heartbeatedSinceFailover = false;
blockContentsStale = true;
}
public void receivedBlockReport() {
if (heartbeatedSinceFailover) {
blockContentsStale = false;
}
firstBlockReport = false;
}
boolean isFirstBlockReport() {
return firstBlockReport;
}
@Override
public String dumpDatanode() {
StringBuilder sb = new StringBuilder(super.dumpDatanode());
@ -606,4 +562,18 @@ public String dumpDatanode() {
}
return sb.toString();
}
DatanodeStorageInfo updateStorage(DatanodeStorage s) {
synchronized (storageMap) {
DatanodeStorageInfo storage = storageMap.get(s.getStorageID());
if (storage == null) {
LOG.info("Adding new storage ID " + s.getStorageID() +
" for DN " + getXferAddr());
storage = new DatanodeStorageInfo(this, s);
storageMap.put(s.getStorageID(), storage);
}
return storage;
}
}
}

View File

@ -403,9 +403,13 @@ DatanodeDescriptor getDatanodeDescriptor(String address) {
}
/** Get a datanode descriptor given corresponding storageID */
DatanodeDescriptor getDatanode(final String storageID) {
return datanodeMap.get(storageID);
/** Get a datanode descriptor given corresponding DatanodeUUID */
DatanodeDescriptor getDatanode(final String datanodeUuid) {
if (datanodeUuid == null) {
return null;
}
return datanodeMap.get(datanodeUuid);
}
/**
@ -417,7 +421,7 @@ DatanodeDescriptor getDatanode(final String storageID) {
*/
public DatanodeDescriptor getDatanode(DatanodeID nodeID
) throws UnregisteredNodeException {
final DatanodeDescriptor node = getDatanode(nodeID.getStorageID());
final DatanodeDescriptor node = getDatanode(nodeID.getDatanodeUuid());
if (node == null)
return null;
if (!node.getXferAddr().equals(nodeID.getXferAddr())) {
@ -430,6 +434,20 @@ public DatanodeDescriptor getDatanode(DatanodeID nodeID
return node;
}
public DatanodeStorageInfo[] getDatanodeStorageInfos(
DatanodeID[] datanodeID, String[] storageIDs)
throws UnregisteredNodeException {
if (datanodeID.length == 0) {
return null;
}
final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length];
for(int i = 0; i < datanodeID.length; i++) {
final DatanodeDescriptor dd = getDatanode(datanodeID[i]);
storages[i] = dd.getStorageInfo(storageIDs[i]);
}
return storages;
}
/** Prints information about all datanodes. */
void datanodeDump(final PrintWriter out) {
synchronized (datanodeMap) {
@ -507,7 +525,7 @@ void addDatanode(final DatanodeDescriptor node) {
// remove from host2DatanodeMap the datanodeDescriptor removed
// from datanodeMap before adding node to host2DatanodeMap.
synchronized(datanodeMap) {
host2DatanodeMap.remove(datanodeMap.put(node.getStorageID(), node));
host2DatanodeMap.remove(datanodeMap.put(node.getDatanodeUuid(), node));
}
networktopology.add(node); // may throw InvalidTopologyException
@ -522,7 +540,7 @@ void addDatanode(final DatanodeDescriptor node) {
/** Physically remove node from datanodeMap. */
private void wipeDatanode(final DatanodeID node) {
final String key = node.getStorageID();
final String key = node.getDatanodeUuid();
synchronized (datanodeMap) {
host2DatanodeMap.remove(datanodeMap.remove(key));
}
@ -683,8 +701,10 @@ boolean checkDecommissionState(DatanodeDescriptor node) {
/** Start decommissioning the specified datanode. */
private void startDecommission(DatanodeDescriptor node) {
if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
LOG.info("Start Decommissioning " + node + " with " +
node.numBlocks() + " blocks");
for (DatanodeStorageInfo storage : node.getStorageInfos()) {
LOG.info("Start Decommissioning " + node + " " + storage
+ " with " + storage.numBlocks() + " blocks");
}
heartbeatManager.startDecommission(node);
node.decommissioningStatus.setStartTime(now());
@ -706,24 +726,6 @@ void stopDecommission(DatanodeDescriptor node) {
}
}
/**
* Generate new storage ID.
*
* @return unique storage ID
*
* Note: that collisions are still possible if somebody will try
* to bring in a data storage from a different cluster.
*/
private String newStorageID() {
String newID = null;
while(newID == null) {
newID = "DS" + Integer.toString(DFSUtil.getRandom().nextInt());
if (datanodeMap.get(newID) != null)
newID = null;
}
return newID;
}
/**
* Register the given datanode with the namenode. NB: the given
* registration is mutated and given back to the datanode.
@ -762,9 +764,9 @@ public void registerDatanode(DatanodeRegistration nodeReg)
}
NameNode.stateChangeLog.info("BLOCK* registerDatanode: from "
+ nodeReg + " storage " + nodeReg.getStorageID());
+ nodeReg + " storage " + nodeReg.getDatanodeUuid());
DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
DatanodeDescriptor nodeS = getDatanode(nodeReg.getDatanodeUuid());
DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
nodeReg.getIpAddr(), nodeReg.getXferPort());
@ -799,7 +801,7 @@ nodes with its data cleared (or user can just remove the StorageID
*/
NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS
+ " is replaced by " + nodeReg + " with the same storageID "
+ nodeReg.getStorageID());
+ nodeReg.getDatanodeUuid());
}
boolean success = false;
@ -831,20 +833,8 @@ nodes with its data cleared (or user can just remove the StorageID
}
}
return;
}
// this is a new datanode serving a new data storage
if ("".equals(nodeReg.getStorageID())) {
// this data storage has never been registered
// it is either empty or was created by pre-storageID version of DFS
nodeReg.setStorageID(newStorageID());
if (NameNode.stateChangeLog.isDebugEnabled()) {
NameNode.stateChangeLog.debug(
"BLOCK* NameSystem.registerDatanode: "
+ "new storageID " + nodeReg.getStorageID() + " assigned.");
}
}
DatanodeDescriptor nodeDescr
= new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK);
boolean success = false;
@ -1212,8 +1202,7 @@ private void setDatanodeDead(DatanodeDescriptor node) {
/** Handle heartbeat from datanodes. */
public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
final String blockPoolId,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
StorageReport[] reports, final String blockPoolId,
int xceiverCount, int maxTransfers, int failedVolumes
) throws IOException {
synchronized (heartbeatManager) {
@ -1235,8 +1224,8 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
return new DatanodeCommand[]{RegisterCommand.REGISTER};
}
heartbeatManager.updateHeartbeat(nodeinfo, capacity, dfsUsed,
remaining, blockPoolUsed, xceiverCount, failedVolumes);
heartbeatManager.updateHeartbeat(nodeinfo, reports,
xceiverCount, failedVolumes);
// If we are in safemode, do not send back any recovery / replication
// requests. Don't even drain the existing queue of work.
@ -1251,32 +1240,32 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
blocks.length);
for (BlockInfoUnderConstruction b : blocks) {
DatanodeDescriptor[] expectedLocations = b.getExpectedLocations();
final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
// Skip stale nodes during recovery - not heart beated for some time (30s by default).
List<DatanodeDescriptor> recoveryLocations =
new ArrayList<DatanodeDescriptor>(expectedLocations.length);
for (int i = 0; i < expectedLocations.length; i++) {
if (!expectedLocations[i].isStale(this.staleInterval)) {
recoveryLocations.add(expectedLocations[i]);
final List<DatanodeStorageInfo> recoveryLocations =
new ArrayList<DatanodeStorageInfo>(storages.length);
for (int i = 0; i < storages.length; i++) {
if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
recoveryLocations.add(storages[i]);
}
}
// If we only get 1 replica after eliminating stale nodes, then choose all
// replicas for recovery and let the primary data node handle failures.
if (recoveryLocations.size() > 1) {
if (recoveryLocations.size() != expectedLocations.length) {
if (recoveryLocations.size() != storages.length) {
LOG.info("Skipped stale nodes for recovery : " +
(expectedLocations.length - recoveryLocations.size()));
(storages.length - recoveryLocations.size()));
}
brCommand.add(new RecoveringBlock(
new ExtendedBlock(blockPoolId, b),
recoveryLocations.toArray(new DatanodeDescriptor[recoveryLocations.size()]),
DatanodeStorageInfo.toDatanodeInfos(recoveryLocations),
b.getBlockRecoveryId()));
} else {
// If too many replicas are stale, then choose all replicas to participate
// in block recovery.
brCommand.add(new RecoveringBlock(
new ExtendedBlock(blockPoolId, b),
expectedLocations,
DatanodeStorageInfo.toDatanodeInfos(storages),
b.getBlockRecoveryId()));
}
}
@ -1342,7 +1331,9 @@ public void markAllDatanodesStale() {
LOG.info("Marking all datandoes as stale");
synchronized (datanodeMap) {
for (DatanodeDescriptor dn : datanodeMap.values()) {
dn.markStaleAfterFailover();
for(DatanodeStorageInfo storage : dn.getStorageInfos()) {
storage.markStaleAfterFailover();
}
}
}
}
@ -1365,3 +1356,4 @@ public String toString() {
return getClass().getSimpleName() + ": " + host2DatanodeMap;
}
}

View File

@ -0,0 +1,288 @@
/**
* 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;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
/**
* A Datanode has one or more storages. A storage in the Datanode is represented
* by this class.
*/
public class DatanodeStorageInfo {
public static final DatanodeStorageInfo[] EMPTY_ARRAY = {};
public static DatanodeInfo[] toDatanodeInfos(DatanodeStorageInfo[] storages) {
return toDatanodeInfos(Arrays.asList(storages));
}
static DatanodeInfo[] toDatanodeInfos(List<DatanodeStorageInfo> storages) {
final DatanodeInfo[] datanodes = new DatanodeInfo[storages.size()];
for(int i = 0; i < storages.size(); i++) {
datanodes[i] = storages.get(i).getDatanodeDescriptor();
}
return datanodes;
}
static DatanodeDescriptor[] toDatanodeDescriptors(
DatanodeStorageInfo[] storages) {
DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length];
for (int i = 0; i < storages.length; ++i) {
datanodes[i] = storages[i].getDatanodeDescriptor();
}
return datanodes;
}
public static String[] toStorageIDs(DatanodeStorageInfo[] storages) {
String[] storageIDs = new String[storages.length];
for(int i = 0; i < storageIDs.length; i++) {
storageIDs[i] = storages[i].getStorageID();
}
return storageIDs;
}
public static StorageType[] toStorageTypes(DatanodeStorageInfo[] storages) {
StorageType[] storageTypes = new StorageType[storages.length];
for(int i = 0; i < storageTypes.length; i++) {
storageTypes[i] = storages[i].getStorageType();
}
return storageTypes;
}
/**
* Iterates over the list of blocks belonging to the data-node.
*/
class BlockIterator implements Iterator<BlockInfo> {
private BlockInfo current;
BlockIterator(BlockInfo head) {
this.current = head;
}
public boolean hasNext() {
return current != null;
}
public BlockInfo next() {
BlockInfo res = current;
current = current.getNext(current.findStorageInfo(DatanodeStorageInfo.this));
return res;
}
public void remove() {
throw new UnsupportedOperationException("Sorry. can't remove.");
}
}
private final DatanodeDescriptor dn;
private final String storageID;
private final StorageType storageType;
private final State state;
private long capacity;
private long dfsUsed;
private long remaining;
private long blockPoolUsed;
private volatile BlockInfo blockList = null;
private int numBlocks = 0;
/** The number of block reports received */
private int blockReportCount = 0;
/**
* Set to false on any NN failover, and reset to true
* whenever a block report is received.
*/
private boolean heartbeatedSinceFailover = false;
/**
* At startup or at failover, the storages in the cluster may have pending
* block deletions from a previous incarnation of the NameNode. The block
* contents are considered as stale until a block report is received. When a
* storage is considered as stale, the replicas on it are also considered as
* stale. If any block has at least one stale replica, then no invalidations
* will be processed for this block. See HDFS-1972.
*/
private boolean blockContentsStale = true;
DatanodeStorageInfo(DatanodeDescriptor dn, DatanodeStorage s) {
this.dn = dn;
this.storageID = s.getStorageID();
this.storageType = s.getStorageType();
this.state = s.getState();
}
int getBlockReportCount() {
return blockReportCount;
}
void setBlockReportCount(int blockReportCount) {
this.blockReportCount = blockReportCount;
}
boolean areBlockContentsStale() {
return blockContentsStale;
}
void markStaleAfterFailover() {
heartbeatedSinceFailover = false;
blockContentsStale = true;
}
void receivedHeartbeat(StorageReport report) {
updateState(report);
heartbeatedSinceFailover = true;
}
void receivedBlockReport() {
if (heartbeatedSinceFailover) {
blockContentsStale = false;
}
blockReportCount++;
}
@VisibleForTesting
public void setUtilizationForTesting(long capacity, long dfsUsed,
long remaining, long blockPoolUsed) {
this.capacity = capacity;
this.dfsUsed = dfsUsed;
this.remaining = remaining;
this.blockPoolUsed = blockPoolUsed;
}
State getState() {
return this.state;
}
String getStorageID() {
return storageID;
}
StorageType getStorageType() {
return storageType;
}
long getCapacity() {
return capacity;
}
long getDfsUsed() {
return dfsUsed;
}
long getRemaining() {
return remaining;
}
long getBlockPoolUsed() {
return blockPoolUsed;
}
boolean addBlock(BlockInfo b) {
if(!b.addStorage(this))
return false;
// add to the head of the data-node list
blockList = b.listInsert(blockList, this);
numBlocks++;
return true;
}
boolean removeBlock(BlockInfo b) {
blockList = b.listRemove(blockList, this);
if (b.removeStorage(this)) {
numBlocks--;
return true;
} else {
return false;
}
}
int numBlocks() {
return numBlocks;
}
Iterator<BlockInfo> getBlockIterator() {
return new BlockIterator(blockList);
}
/**
* Move block to the head of the list of blocks belonging to the data-node.
* @return the index of the head of the blockList
*/
int moveBlockToHead(BlockInfo b, int curIndex, int headIndex) {
blockList = b.moveBlockToHead(blockList, this, curIndex, headIndex);
return curIndex;
}
/**
* Used for testing only
* @return the head of the blockList
*/
@VisibleForTesting
BlockInfo getBlockListHeadForTesting(){
return blockList;
}
void updateState(StorageReport r) {
capacity = r.getCapacity();
dfsUsed = r.getDfsUsed();
remaining = r.getRemaining();
blockPoolUsed = r.getBlockPoolUsed();
}
public DatanodeDescriptor getDatanodeDescriptor() {
return dn;
}
/** Increment the number of blocks scheduled for each given storage */
public static void incrementBlocksScheduled(DatanodeStorageInfo... storages) {
for (DatanodeStorageInfo s : storages) {
s.getDatanodeDescriptor().incrementBlocksScheduled();
}
}
@Override
public boolean equals(Object obj) {
if (this == obj) {
return true;
} else if (obj == null || !(obj instanceof DatanodeStorageInfo)) {
return false;
}
final DatanodeStorageInfo that = (DatanodeStorageInfo)obj;
return this.storageID.equals(that.storageID);
}
@Override
public int hashCode() {
return storageID.hashCode();
}
@Override
public String toString() {
return "[" + storageType + "]" + storageID + ":" + state;
}
}

View File

@ -27,6 +27,7 @@
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.Time;
@ -170,7 +171,7 @@ synchronized void register(final DatanodeDescriptor d) {
addDatanode(d);
//update its timestamp
d.updateHeartbeat(0L, 0L, 0L, 0L, 0, 0);
d.updateHeartbeat(StorageReport.EMPTY_ARRAY, 0, 0);
}
}
@ -192,11 +193,9 @@ synchronized void removeDatanode(DatanodeDescriptor node) {
}
synchronized void updateHeartbeat(final DatanodeDescriptor node,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
int xceiverCount, int failedVolumes) {
StorageReport[] reports, int xceiverCount, int failedVolumes) {
stats.subtract(node);
node.updateHeartbeat(capacity, dfsUsed, remaining, blockPoolUsed,
xceiverCount, failedVolumes);
node.updateHeartbeat(reports, xceiverCount, failedVolumes);
stats.add(node);
}
@ -341,3 +340,4 @@ private void incrExpiredHeartbeats() {
}
}
}

View File

@ -80,10 +80,10 @@ synchronized boolean contains(final String storageID, final Block block) {
*/
synchronized void add(final Block block, final DatanodeInfo datanode,
final boolean log) {
LightWeightHashSet<Block> set = node2blocks.get(datanode.getStorageID());
LightWeightHashSet<Block> set = node2blocks.get(datanode.getDatanodeUuid());
if (set == null) {
set = new LightWeightHashSet<Block>();
node2blocks.put(datanode.getStorageID(), set);
node2blocks.put(datanode.getDatanodeUuid(), set);
}
if (set.add(block)) {
numBlocks++;

View File

@ -34,5 +34,5 @@ public interface MutableBlockCollection extends BlockCollection {
* and set the locations.
*/
public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeDescriptor[] locations) throws IOException;
DatanodeStorageInfo[] storages) throws IOException;
}

View File

@ -42,11 +42,13 @@ class PendingDataNodeMessages {
static class ReportedBlockInfo {
private final Block block;
private final DatanodeDescriptor dn;
private final String storageID;
private final ReplicaState reportedState;
ReportedBlockInfo(DatanodeDescriptor dn, Block block,
ReportedBlockInfo(DatanodeDescriptor dn, String storageID, Block block,
ReplicaState reportedState) {
this.dn = dn;
this.storageID = storageID;
this.block = block;
this.reportedState = reportedState;
}
@ -58,6 +60,10 @@ Block getBlock() {
DatanodeDescriptor getNode() {
return dn;
}
String getStorageID() {
return storageID;
}
ReplicaState getReportedState() {
return reportedState;
@ -70,11 +76,11 @@ public String toString() {
}
}
void enqueueReportedBlock(DatanodeDescriptor dn, Block block,
void enqueueReportedBlock(DatanodeDescriptor dn, String storageID, Block block,
ReplicaState reportedState) {
block = new Block(block);
getBlockQueue(block).add(
new ReportedBlockInfo(dn, block, reportedState));
new ReportedBlockInfo(dn, storageID, block, reportedState));
count++;
}

View File

@ -236,6 +236,8 @@ public static class StorageDirectory implements FormatConfirmable {
final boolean useLock; // flag to enable storage lock
final StorageDirType dirType; // storage dir type
FileLock lock; // storage lock
private String storageUuid = null; // Storage directory identifier.
public StorageDirectory(File dir) {
// default dirType is null
@ -246,6 +248,14 @@ public StorageDirectory(File dir, StorageDirType dirType) {
this(dir, dirType, true);
}
public void setStorageUuid(String storageUuid) {
this.storageUuid = storageUuid;
}
public String getStorageUuid() {
return storageUuid;
}
/**
* Constructor
* @param dir directory corresponding to the storage

View File

@ -27,6 +27,7 @@
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -159,31 +160,32 @@ synchronized String getBlockPoolId() {
synchronized NamespaceInfo getNamespaceInfo() {
return bpNSInfo;
}
@Override
public synchronized String toString() {
if (bpNSInfo == null) {
// If we haven't yet connected to our NN, we don't yet know our
// own block pool ID.
// If _none_ of the block pools have connected yet, we don't even
// know the storage ID of this DN.
String storageId = dn.getStorageId();
if (storageId == null || "".equals(storageId)) {
storageId = "unknown";
// know the DatanodeID ID of this DN.
String datanodeUuid = dn.getDatanodeUuid();
if (datanodeUuid == null || datanodeUuid.isEmpty()) {
datanodeUuid = "unassigned";
}
return "Block pool <registering> (storage id " + storageId +
")";
return "Block pool <registering> (Datanode Uuid " + datanodeUuid + ")";
} else {
return "Block pool " + getBlockPoolId() +
" (storage id " + dn.getStorageId() +
")";
" (Datanode Uuid " + dn.getDatanodeUuid() +
")";
}
}
void reportBadBlocks(ExtendedBlock block) {
void reportBadBlocks(ExtendedBlock block,
String storageUuid, StorageType storageType) {
checkBlock(block);
for (BPServiceActor actor : bpServices) {
actor.reportBadBlocks(block);
actor.reportBadBlocks(block, storageUuid, storageType);
}
}
@ -192,7 +194,8 @@ void reportBadBlocks(ExtendedBlock block) {
* till namenode is informed before responding with success to the
* client? For now we don't.
*/
void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
void notifyNamenodeReceivedBlock(
ExtendedBlock block, String delHint, String storageUuid) {
checkBlock(block);
checkDelHint(delHint);
ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
@ -201,7 +204,7 @@ void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
delHint);
for (BPServiceActor actor : bpServices) {
actor.notifyNamenodeBlockImmediately(bInfo);
actor.notifyNamenodeBlockImmediately(bInfo, storageUuid);
}
}
@ -218,23 +221,23 @@ private void checkDelHint(String delHint) {
"delHint is null");
}
void notifyNamenodeDeletedBlock(ExtendedBlock block) {
void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) {
checkBlock(block);
ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
block.getLocalBlock(), BlockStatus.DELETED_BLOCK, null);
for (BPServiceActor actor : bpServices) {
actor.notifyNamenodeDeletedBlock(bInfo);
actor.notifyNamenodeDeletedBlock(bInfo, storageUuid);
}
}
void notifyNamenodeReceivingBlock(ExtendedBlock block) {
void notifyNamenodeReceivingBlock(ExtendedBlock block, String storageUuid) {
checkBlock(block);
ReceivedDeletedBlockInfo bInfo = new ReceivedDeletedBlockInfo(
block.getLocalBlock(), BlockStatus.RECEIVING_BLOCK, null);
for (BPServiceActor actor : bpServices) {
actor.notifyNamenodeBlockImmediately(bInfo);
actor.notifyNamenodeBlockImmediately(bInfo, storageUuid);
}
}
@ -337,7 +340,7 @@ private static void checkNSEquality(
}
}
synchronized DatanodeRegistration createRegistration() {
synchronized DatanodeRegistration createRegistration() throws IOException {
Preconditions.checkState(bpNSInfo != null,
"getRegistration() can only be called after initial handshake");
return dn.createBPRegistration(bpNSInfo);

View File

@ -22,7 +22,6 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.util.Collection;
import java.util.Map;
@ -30,6 +29,7 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@ -51,7 +51,6 @@
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.VersionInfo;
import org.apache.hadoop.util.VersionUtil;
@ -97,9 +96,9 @@ class BPServiceActor implements Runnable {
* keyed by block ID, contains the pending changes which have yet to be
* reported to the NN. Access should be synchronized on this object.
*/
private final Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR
= Maps.newHashMap();
private final Map<String, PerStoragePendingIncrementalBR>
pendingIncrementalBRperStorage = Maps.newHashMap();
private volatile int pendingReceivedRequests = 0;
private volatile boolean shouldServiceRun = true;
private final DataNode dn;
@ -241,12 +240,15 @@ void scheduleBlockReport(long delay) {
resetBlockReportTime = true; // reset future BRs for randomness
}
void reportBadBlocks(ExtendedBlock block) {
void reportBadBlocks(ExtendedBlock block,
String storageUuid, StorageType storageType) {
if (bpRegistration == null) {
return;
}
DatanodeInfo[] dnArr = { new DatanodeInfo(bpRegistration) };
LocatedBlock[] blocks = { new LocatedBlock(block, dnArr) };
String[] uuids = { storageUuid };
StorageType[] types = { storageType };
LocatedBlock[] blocks = { new LocatedBlock(block, dnArr, uuids, types) };
try {
bpNamenode.reportBadBlocks(blocks);
@ -260,49 +262,100 @@ void reportBadBlocks(ExtendedBlock block) {
}
/**
* Report received blocks and delete hints to the Namenode
*
* Report received blocks and delete hints to the Namenode for each
* storage.
*
* @throws IOException
*/
private void reportReceivedDeletedBlocks() throws IOException {
// check if there are newly received blocks
ReceivedDeletedBlockInfo[] receivedAndDeletedBlockArray = null;
synchronized (pendingIncrementalBR) {
int numBlocks = pendingIncrementalBR.size();
if (numBlocks > 0) {
//
// Send newly-received and deleted blockids to namenode
//
receivedAndDeletedBlockArray = pendingIncrementalBR
.values().toArray(new ReceivedDeletedBlockInfo[numBlocks]);
}
pendingIncrementalBR.clear();
}
if (receivedAndDeletedBlockArray != null) {
StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
bpRegistration.getStorageID(), receivedAndDeletedBlockArray) };
boolean success = false;
try {
bpNamenode.blockReceivedAndDeleted(bpRegistration, bpos.getBlockPoolId(),
report);
success = true;
} finally {
synchronized (pendingIncrementalBR) {
if (!success) {
// If we didn't succeed in sending the report, put all of the
// blocks back onto our queue, but only in the case where we didn't
// put something newer in the meantime.
for (ReceivedDeletedBlockInfo rdbi : receivedAndDeletedBlockArray) {
if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
}
}
}
pendingReceivedRequests = pendingIncrementalBR.size();
// Generate a list of the pending reports for each storage under the lock
Map<String, ReceivedDeletedBlockInfo[]> blockArrays = Maps.newHashMap();
synchronized (pendingIncrementalBRperStorage) {
for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
pendingIncrementalBRperStorage.entrySet()) {
final String storageUuid = entry.getKey();
final PerStoragePendingIncrementalBR perStorageMap = entry.getValue();
if (perStorageMap.getBlockInfoCount() > 0) {
// Send newly-received and deleted blockids to namenode
ReceivedDeletedBlockInfo[] rdbi = perStorageMap.dequeueBlockInfos();
pendingReceivedRequests =
(pendingReceivedRequests > rdbi.length ?
(pendingReceivedRequests - rdbi.length) : 0);
blockArrays.put(storageUuid, rdbi);
}
}
}
// Send incremental block reports to the Namenode outside the lock
for (Map.Entry<String, ReceivedDeletedBlockInfo[]> entry :
blockArrays.entrySet()) {
final String storageUuid = entry.getKey();
final ReceivedDeletedBlockInfo[] rdbi = entry.getValue();
StorageReceivedDeletedBlocks[] report = { new StorageReceivedDeletedBlocks(
storageUuid, rdbi) };
boolean success = false;
try {
bpNamenode.blockReceivedAndDeleted(bpRegistration,
bpos.getBlockPoolId(), report);
success = true;
} finally {
if (!success) {
synchronized (pendingIncrementalBRperStorage) {
// If we didn't succeed in sending the report, put all of the
// blocks back onto our queue, but only in the case where we
// didn't put something newer in the meantime.
PerStoragePendingIncrementalBR perStorageMap =
pendingIncrementalBRperStorage.get(storageUuid);
pendingReceivedRequests += perStorageMap.putMissingBlockInfos(rdbi);
}
}
}
}
}
/**
* Retrieve the incremental BR state for a given storage UUID
* @param storageUuid
* @return
*/
private PerStoragePendingIncrementalBR getIncrementalBRMapForStorage(
String storageUuid) {
PerStoragePendingIncrementalBR mapForStorage =
pendingIncrementalBRperStorage.get(storageUuid);
if (mapForStorage == null) {
// This is the first time we are adding incremental BR state for
// this storage so create a new map. This is required once per
// storage, per service actor.
mapForStorage = new PerStoragePendingIncrementalBR();
pendingIncrementalBRperStorage.put(storageUuid, mapForStorage);
}
return mapForStorage;
}
/**
* Add a blockInfo for notification to NameNode. If another entry
* exists for the same block it is removed.
*
* Caller must synchronize access using pendingIncrementalBRperStorage.
* @param bInfo
* @param storageUuid
*/
void addPendingReplicationBlockInfo(ReceivedDeletedBlockInfo bInfo,
String storageUuid) {
// Make sure another entry for the same block is first removed.
// There may only be one such entry.
for (Map.Entry<String, PerStoragePendingIncrementalBR> entry :
pendingIncrementalBRperStorage.entrySet()) {
if (entry.getValue().removeBlockInfo(bInfo)) {
break;
}
}
getIncrementalBRMapForStorage(storageUuid).putBlockInfo(bInfo);
}
/*
@ -310,19 +363,19 @@ private void reportReceivedDeletedBlocks() throws IOException {
* till namenode is informed before responding with success to the
* client? For now we don't.
*/
void notifyNamenodeBlockImmediately(ReceivedDeletedBlockInfo bInfo) {
synchronized (pendingIncrementalBR) {
pendingIncrementalBR.put(
bInfo.getBlock().getBlockId(), bInfo);
void notifyNamenodeBlockImmediately(
ReceivedDeletedBlockInfo bInfo, String storageUuid) {
synchronized (pendingIncrementalBRperStorage) {
addPendingReplicationBlockInfo(bInfo, storageUuid);
pendingReceivedRequests++;
pendingIncrementalBR.notifyAll();
pendingIncrementalBRperStorage.notifyAll();
}
}
void notifyNamenodeDeletedBlock(ReceivedDeletedBlockInfo bInfo) {
synchronized (pendingIncrementalBR) {
pendingIncrementalBR.put(
bInfo.getBlock().getBlockId(), bInfo);
void notifyNamenodeDeletedBlock(
ReceivedDeletedBlockInfo bInfo, String storageUuid) {
synchronized (pendingIncrementalBRperStorage) {
addPendingReplicationBlockInfo(bInfo, storageUuid);
}
}
@ -331,13 +384,13 @@ void notifyNamenodeDeletedBlock(ReceivedDeletedBlockInfo bInfo) {
*/
@VisibleForTesting
void triggerBlockReportForTests() {
synchronized (pendingIncrementalBR) {
synchronized (pendingIncrementalBRperStorage) {
lastBlockReport = 0;
lastHeartbeat = 0;
pendingIncrementalBR.notifyAll();
pendingIncrementalBRperStorage.notifyAll();
while (lastBlockReport == 0) {
try {
pendingIncrementalBR.wait(100);
pendingIncrementalBRperStorage.wait(100);
} catch (InterruptedException e) {
return;
}
@ -347,12 +400,12 @@ void triggerBlockReportForTests() {
@VisibleForTesting
void triggerHeartbeatForTests() {
synchronized (pendingIncrementalBR) {
synchronized (pendingIncrementalBRperStorage) {
lastHeartbeat = 0;
pendingIncrementalBR.notifyAll();
pendingIncrementalBRperStorage.notifyAll();
while (lastHeartbeat == 0) {
try {
pendingIncrementalBR.wait(100);
pendingIncrementalBRperStorage.wait(100);
} catch (InterruptedException e) {
return;
}
@ -362,13 +415,13 @@ void triggerHeartbeatForTests() {
@VisibleForTesting
void triggerDeletionReportForTests() {
synchronized (pendingIncrementalBR) {
synchronized (pendingIncrementalBRperStorage) {
lastDeletedReport = 0;
pendingIncrementalBR.notifyAll();
pendingIncrementalBRperStorage.notifyAll();
while (lastDeletedReport == 0) {
try {
pendingIncrementalBR.wait(100);
pendingIncrementalBRperStorage.wait(100);
} catch (InterruptedException e) {
return;
}
@ -392,23 +445,38 @@ DatanodeCommand blockReport() throws IOException {
// a FINALIZED one.
reportReceivedDeletedBlocks();
// Send one block report per known storage.
// Create block report
long brCreateStartTime = now();
BlockListAsLongs bReport = dn.getFSDataset().getBlockReport(
bpos.getBlockPoolId());
long totalBlockCount = 0;
Map<DatanodeStorage, BlockListAsLongs> perVolumeBlockLists =
dn.getFSDataset().getBlockReports(bpos.getBlockPoolId());
// Send block report
long brSendStartTime = now();
StorageBlockReport[] report = { new StorageBlockReport(
new DatanodeStorage(bpRegistration.getStorageID()),
bReport.getBlockListAsLongs()) };
cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), report);
StorageBlockReport[] reports =
new StorageBlockReport[perVolumeBlockLists.size()];
int i = 0;
for(Map.Entry<DatanodeStorage, BlockListAsLongs> kvPair : perVolumeBlockLists.entrySet()) {
DatanodeStorage dnStorage = kvPair.getKey();
BlockListAsLongs blockList = kvPair.getValue();
totalBlockCount += blockList.getNumberOfBlocks();
reports[i++] =
new StorageBlockReport(
dnStorage, blockList.getBlockListAsLongs());
}
cmd = bpNamenode.blockReport(bpRegistration, bpos.getBlockPoolId(), reports);
// Log the block report processing stats from Datanode perspective
long brSendCost = now() - brSendStartTime;
long brCreateCost = brSendStartTime - brCreateStartTime;
dn.getMetrics().addBlockReport(brSendCost);
LOG.info("BlockReport of " + bReport.getNumberOfBlocks()
LOG.info("BlockReport of " + totalBlockCount
+ " blocks took " + brCreateCost + " msec to generate and "
+ brSendCost + " msecs for RPC and NN processing");
@ -434,17 +502,15 @@ DatanodeCommand blockReport() throws IOException {
HeartbeatResponse sendHeartBeat() throws IOException {
StorageReport[] reports =
dn.getFSDataset().getStorageReports(bpos.getBlockPoolId());
if (LOG.isDebugEnabled()) {
LOG.debug("Sending heartbeat from service actor: " + this);
LOG.debug("Sending heartbeat with " + reports.length +
" storage reports from service actor: " + this);
}
// reports number of failed volumes
StorageReport[] report = { new StorageReport(bpRegistration.getStorageID(),
false,
dn.getFSDataset().getCapacity(),
dn.getFSDataset().getDfsUsed(),
dn.getFSDataset().getRemaining(),
dn.getFSDataset().getBlockPoolUsed(bpos.getBlockPoolId())) };
return bpNamenode.sendHeartbeat(bpRegistration, report,
return bpNamenode.sendHeartbeat(bpRegistration,
reports,
dn.getXmitsInProgress(),
dn.getXceiverCount(),
dn.getFSDataset().getNumFailedVolumes());
@ -462,9 +528,9 @@ void start() {
}
private String formatThreadName() {
Collection<URI> dataDirs = DataNode.getStorageDirs(dn.getConf());
return "DataNode: [" +
StringUtils.uriToString(dataDirs.toArray(new URI[0])) + "] " +
Collection<StorageLocation> dataDirs =
DataNode.getStorageLocations(dn.getConf());
return "DataNode: [" + dataDirs.toString() + "] " +
" heartbeating to " + nnAddr;
}
@ -570,10 +636,10 @@ private void offerService() throws Exception {
//
long waitTime = dnConf.heartBeatInterval -
(Time.now() - lastHeartbeat);
synchronized(pendingIncrementalBR) {
synchronized(pendingIncrementalBRperStorage) {
if (waitTime > 0 && pendingReceivedRequests == 0) {
try {
pendingIncrementalBR.wait(waitTime);
pendingIncrementalBRperStorage.wait(waitTime);
} catch (InterruptedException ie) {
LOG.warn("BPOfferService for " + this + " interrupted");
}
@ -744,4 +810,68 @@ void reRegister() throws IOException {
}
}
private static class PerStoragePendingIncrementalBR {
private Map<Long, ReceivedDeletedBlockInfo> pendingIncrementalBR =
Maps.newHashMap();
/**
* Return the number of blocks on this storage that have pending
* incremental block reports.
* @return
*/
int getBlockInfoCount() {
return pendingIncrementalBR.size();
}
/**
* Dequeue and return all pending incremental block report state.
* @return
*/
ReceivedDeletedBlockInfo[] dequeueBlockInfos() {
ReceivedDeletedBlockInfo[] blockInfos =
pendingIncrementalBR.values().toArray(
new ReceivedDeletedBlockInfo[getBlockInfoCount()]);
pendingIncrementalBR.clear();
return blockInfos;
}
/**
* Add blocks from blockArray to pendingIncrementalBR, unless the
* block already exists in pendingIncrementalBR.
* @param blockArray list of blocks to add.
* @return the number of missing blocks that we added.
*/
int putMissingBlockInfos(ReceivedDeletedBlockInfo[] blockArray) {
int blocksPut = 0;
for (ReceivedDeletedBlockInfo rdbi : blockArray) {
if (!pendingIncrementalBR.containsKey(rdbi.getBlock().getBlockId())) {
pendingIncrementalBR.put(rdbi.getBlock().getBlockId(), rdbi);
++blocksPut;
}
}
return blocksPut;
}
/**
* Add pending incremental block report for a single block.
* @param blockID
* @param blockInfo
*/
void putBlockInfo(ReceivedDeletedBlockInfo blockInfo) {
pendingIncrementalBR.put(blockInfo.getBlock().getBlockId(), blockInfo);
}
/**
* Remove pending incremental block report for a single block if it
* exists.
*
* @param blockInfo
* @return true if a report was removed, false if no report existed for
* the given block.
*/
boolean removeBlockInfo(ReceivedDeletedBlockInfo blockInfo) {
return (pendingIncrementalBR.remove(blockInfo.getBlock().getBlockId()) != null);
}
}
}

View File

@ -187,7 +187,7 @@ public LinkedElement getNext() {
+ hours + " hours for block pool " + bpid);
// get the list of blocks and arrange them in random order
List<Block> arr = dataset.getFinalizedBlocks(blockPoolId);
List<FinalizedReplica> arr = dataset.getFinalizedBlocks(blockPoolId);
Collections.shuffle(arr);
long scanTime = -1;

View File

@ -162,7 +162,8 @@ class BlockReceiver implements Closeable {
switch (stage) {
case PIPELINE_SETUP_CREATE:
replicaInfo = datanode.data.createRbw(block);
datanode.notifyNamenodeReceivingBlock(block);
datanode.notifyNamenodeReceivingBlock(
block, replicaInfo.getStorageUuid());
break;
case PIPELINE_SETUP_STREAMING_RECOVERY:
replicaInfo = datanode.data.recoverRbw(
@ -176,7 +177,8 @@ class BlockReceiver implements Closeable {
block.getLocalBlock());
}
block.setGenerationStamp(newGs);
datanode.notifyNamenodeReceivingBlock(block);
datanode.notifyNamenodeReceivingBlock(
block, replicaInfo.getStorageUuid());
break;
case PIPELINE_SETUP_APPEND_RECOVERY:
replicaInfo = datanode.data.recoverAppend(block, newGs, minBytesRcvd);
@ -185,7 +187,8 @@ class BlockReceiver implements Closeable {
block.getLocalBlock());
}
block.setGenerationStamp(newGs);
datanode.notifyNamenodeReceivingBlock(block);
datanode.notifyNamenodeReceivingBlock(
block, replicaInfo.getStorageUuid());
break;
case TRANSFER_RBW:
case TRANSFER_FINALIZED:
@ -252,6 +255,10 @@ class BlockReceiver implements Closeable {
/** Return the datanode object. */
DataNode getDataNode() {return datanode;}
String getStorageUuid() {
return replicaInfo.getStorageUuid();
}
/**
* close files.
*/
@ -1027,7 +1034,7 @@ public void run() {
final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
block.setNumBytes(replicaInfo.getNumBytes());
datanode.data.finalizeBlock(block);
datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT, replicaInfo.getStorageUuid());
if (ClientTraceLog.isInfoEnabled() && isClient) {
long offset = 0;
DatanodeRegistration dnR =
@ -1035,7 +1042,7 @@ public void run() {
ClientTraceLog.info(String.format(DN_CLIENTTRACE_FORMAT,
inAddr, myAddr, block.getNumBytes(),
"HDFS_WRITE", clientname, offset,
dnR.getStorageID(), block, endTime-startTime));
dnR.getDatanodeUuid(), block, endTime-startTime));
} else {
LOG.info("Received " + block + " size "
+ block.getNumBytes() + " from " + inAddr);

View File

@ -17,10 +17,40 @@
*/
package org.apache.hadoop.hdfs.server.datanode;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.protobuf.BlockingService;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.util.ExitUtil.terminate;
import java.io.BufferedOutputStream;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.net.Socket;
import java.net.SocketException;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.net.UnknownHostException;
import java.nio.channels.ClosedByInterruptException;
import java.nio.channels.SocketChannel;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicInteger;
import javax.management.ObjectName;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -39,21 +69,42 @@
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.net.DomainPeerServer;
import org.apache.hadoop.hdfs.net.TcpPeerServer;
import org.apache.hadoop.hdfs.protocol.*;
import org.apache.hadoop.hdfs.protocol.datatransfer.*;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.ClientDatanodeProtocolProtos.ClientDatanodeProtocolService;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DNTransferAckProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
import org.apache.hadoop.hdfs.protocol.proto.InterDatanodeProtocolProtos.InterDatanodeProtocolService;
import org.apache.hadoop.hdfs.protocolPB.*;
import org.apache.hadoop.hdfs.security.token.block.*;
import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.ClientDatanodeProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolPB;
import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.InterDatanodeProtocolTranslatorPB;
import org.apache.hadoop.hdfs.protocolPB.PBHelper;
import org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.common.Util;
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@ -62,7 +113,11 @@
import org.apache.hadoop.hdfs.server.namenode.FileChecksumServlets;
import org.apache.hadoop.hdfs.server.namenode.StreamFile;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.*;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
import org.apache.hadoop.hdfs.web.resources.Param;
import org.apache.hadoop.http.HttpConfig;
@ -84,23 +139,21 @@
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.util.*;
import org.apache.hadoop.util.Daemon;
import org.apache.hadoop.util.DiskChecker;
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.JvmPauseMonitor;
import org.apache.hadoop.util.ServicePlugin;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.VersionInfo;
import org.mortbay.util.ajax.JSON;
import java.io.*;
import java.net.*;
import java.nio.channels.ClosedByInterruptException;
import java.nio.channels.SocketChannel;
import java.security.PrivilegedExceptionAction;
import java.util.*;
import java.util.concurrent.atomic.AtomicInteger;
import javax.management.ObjectName;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.util.ExitUtil.terminate;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.protobuf.BlockingService;
/**********************************************************
* DataNode is a class (and program) that stores a set of
@ -209,7 +262,7 @@ public static InetSocketAddress createSocketAddr(String target) {
private JvmPauseMonitor pauseMonitor;
private SecureResources secureResources = null;
private AbstractList<File> dataDirs;
private List<StorageLocation> dataDirs;
private Configuration conf;
private final List<String> usersWithLocalPathAccess;
@ -218,21 +271,12 @@ public static InetSocketAddress createSocketAddr(String target) {
private final boolean getHdfsBlockLocationsEnabled;
private ObjectName dataNodeInfoBeanName;
/**
* Create the DataNode given a configuration and an array of dataDirs.
* 'dataDirs' is where the blocks are stored.
*/
DataNode(final Configuration conf,
final AbstractList<File> dataDirs) throws IOException {
this(conf, dataDirs, null);
}
/**
* Create the DataNode given a configuration, an array of dataDirs,
* and a namenode proxy
*/
DataNode(final Configuration conf,
final AbstractList<File> dataDirs,
DataNode(final Configuration conf,
final List<StorageLocation> dataDirs,
final SecureResources resources) throws IOException {
super(conf);
@ -491,7 +535,7 @@ private synchronized void initDirectoryScanner(Configuration conf) {
directoryScanner.start();
} else {
LOG.info("Periodic Directory Tree Verification scan is disabled because " +
reason);
reason);
}
}
@ -563,10 +607,11 @@ static DomainPeerServer getDomainPeerServer(Configuration conf,
}
// calls specific to BP
protected void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint) {
protected void notifyNamenodeReceivedBlock(
ExtendedBlock block, String delHint, String storageUuid) {
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos != null) {
bpos.notifyNamenodeReceivedBlock(block, delHint);
bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
} else {
LOG.error("Cannot find BPOfferService for reporting block received for bpid="
+ block.getBlockPoolId());
@ -574,10 +619,11 @@ protected void notifyNamenodeReceivedBlock(ExtendedBlock block, String delHint)
}
// calls specific to BP
protected void notifyNamenodeReceivingBlock(ExtendedBlock block) {
protected void notifyNamenodeReceivingBlock(
ExtendedBlock block, String storageUuid) {
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos != null) {
bpos.notifyNamenodeReceivingBlock(block);
bpos.notifyNamenodeReceivingBlock(block, storageUuid);
} else {
LOG.error("Cannot find BPOfferService for reporting block receiving for bpid="
+ block.getBlockPoolId());
@ -585,10 +631,10 @@ protected void notifyNamenodeReceivingBlock(ExtendedBlock block) {
}
/** Notify the corresponding namenode to delete the block. */
public void notifyNamenodeDeletedBlock(ExtendedBlock block) {
public void notifyNamenodeDeletedBlock(ExtendedBlock block, String storageUuid) {
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if (bpos != null) {
bpos.notifyNamenodeDeletedBlock(block);
bpos.notifyNamenodeDeletedBlock(block, storageUuid);
} else {
LOG.error("Cannot find BPOfferService for reporting block deleted for bpid="
+ block.getBlockPoolId());
@ -600,7 +646,9 @@ public void notifyNamenodeDeletedBlock(ExtendedBlock block) {
*/
public void reportBadBlocks(ExtendedBlock block) throws IOException{
BPOfferService bpos = getBPOSForBlock(block);
bpos.reportBadBlocks(block);
FsVolumeSpi volume = getFSDataset().getVolume(block);
bpos.reportBadBlocks(
block, volume.getStorageID(), volume.getStorageType());
}
/**
@ -672,7 +720,7 @@ boolean areHeartbeatsDisabledForTests() {
* @throws IOException
*/
void startDataNode(Configuration conf,
AbstractList<File> dataDirs,
List<StorageLocation> dataDirs,
// DatanodeProtocol namenode,
SecureResources resources
) throws IOException {
@ -712,19 +760,42 @@ void startDataNode(Configuration conf,
readaheadPool = ReadaheadPool.getInstance();
}
public static String generateUuid() {
return UUID.randomUUID().toString();
}
/**
* Verify that the DatanodeUuid has been initialized. If this is a new
* datanode then we generate a new Datanode Uuid and persist it to disk.
*
* @throws IOException
*/
private synchronized void checkDatanodeUuid() throws IOException {
if (storage.getDatanodeUuid() == null) {
storage.setDatanodeUuid(generateUuid());
storage.writeAll();
LOG.info("Generated and persisted new Datanode UUID " +
storage.getDatanodeUuid());
}
}
/**
* Create a DatanodeRegistration for a specific block pool.
* @param nsInfo the namespace info from the first part of the NN handshake
*/
DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo) {
DatanodeRegistration createBPRegistration(NamespaceInfo nsInfo)
throws IOException {
StorageInfo storageInfo = storage.getBPStorage(nsInfo.getBlockPoolID());
if (storageInfo == null) {
// it's null in the case of SimulatedDataSet
storageInfo = new StorageInfo(nsInfo);
}
checkDatanodeUuid();
DatanodeID dnId = new DatanodeID(
streamingAddr.getAddress().getHostAddress(), hostName,
getStorageId(), getXferPort(), getInfoPort(),
storage.getDatanodeUuid(), getXferPort(), getInfoPort(),
infoSecurePort, getIpcPort());
return new DatanodeRegistration(dnId, storageInfo,
new ExportedBlockKeys(), VersionInfo.getVersion());
@ -743,16 +814,10 @@ synchronized void bpRegistrationSucceeded(DatanodeRegistration bpRegistration,
id = bpRegistration;
}
if (storage.getStorageID().equals("")) {
// This is a fresh datanode, persist the NN-provided storage ID
storage.setStorageID(bpRegistration.getStorageID());
storage.writeAll();
LOG.info("New storage id " + bpRegistration.getStorageID()
+ " is assigned to data-node " + bpRegistration);
} else if(!storage.getStorageID().equals(bpRegistration.getStorageID())) {
throw new IOException("Inconsistent storage IDs. Name-node returned "
+ bpRegistration.getStorageID()
+ ". Expecting " + storage.getStorageID());
if(!storage.getDatanodeUuid().equals(bpRegistration.getDatanodeUuid())) {
throw new IOException("Inconsistent Datanode IDs. Name-node returned "
+ bpRegistration.getDatanodeUuid()
+ ". Expecting " + storage.getDatanodeUuid());
}
registerBlockPoolWithSecretManager(bpRegistration, blockPoolId);
@ -873,7 +938,7 @@ private void initStorage(final NamespaceInfo nsInfo) throws IOException {
final StorageInfo bpStorage = storage.getBPStorage(bpid);
LOG.info("Setting up storage: nsid=" + bpStorage.getNamespaceID()
+ ";bpid=" + bpid + ";lv=" + storage.getLayoutVersion()
+ ";nsInfo=" + nsInfo);
+ ";nsInfo=" + nsInfo + ";dnuuid=" + storage.getDatanodeUuid());
}
synchronized(this) {
@ -900,10 +965,6 @@ public int getXferPort() {
return streamingAddr.getPort();
}
String getStorageId() {
return storage.getStorageID();
}
/**
* @return name useful for logging
*/
@ -989,34 +1050,6 @@ DataNodeMetrics getMetrics() {
return metrics;
}
public static void setNewStorageID(DatanodeID dnId) {
LOG.info("Datanode is " + dnId);
dnId.setStorageID(createNewStorageId(dnId.getXferPort()));
}
/**
* @return a unique storage ID of form "DS-randInt-ipaddr-port-timestamp"
*/
static String createNewStorageId(int port) {
// It is unlikely that we will create a non-unique storage ID
// for the following reasons:
// a) SecureRandom is a cryptographically strong random number generator
// b) IP addresses will likely differ on different hosts
// c) DataNode xfer ports will differ on the same host
// d) StorageIDs will likely be generated at different times (in ms)
// A conflict requires that all four conditions are violated.
// NB: The format of this string can be changed in the future without
// requiring that old SotrageIDs be updated.
String ip = "unknownIP";
try {
ip = DNS.getDefaultIP("default");
} catch (UnknownHostException ignored) {
LOG.warn("Could not find an IP address for the \"default\" inteface.");
}
int rand = DFSUtil.getSecureRandom().nextInt(Integer.MAX_VALUE);
return "DS-" + rand + "-" + ip + "-" + port + "-" + Time.now();
}
/** Ensure the authentication method is kerberos */
private void checkKerberosAuthMethod(String msg) throws IOException {
// User invoking the call must be same as the datanode user
@ -1342,8 +1375,10 @@ private void transferBlock(ExtendedBlock block, DatanodeInfo xferTargets[])
// Check if NN recorded length matches on-disk length
long onDiskLength = data.getLength(block);
if (block.getNumBytes() > onDiskLength) {
FsVolumeSpi volume = getFSDataset().getVolume(block);
// Shorter on-disk len indicates corruption so report NN the corrupt block
bpos.reportBadBlocks(block);
bpos.reportBadBlocks(
block, volume.getStorageID(), volume.getStorageType());
LOG.warn("Can't replicate block " + block
+ " because on-disk length " + onDiskLength
+ " is shorter than NameNode recorded length " + block.getNumBytes());
@ -1607,11 +1642,11 @@ public void run() {
* @param block
* @param delHint
*/
void closeBlock(ExtendedBlock block, String delHint) {
void closeBlock(ExtendedBlock block, String delHint, String storageUuid) {
metrics.incrBlocksWritten();
BPOfferService bpos = blockPoolManager.get(block.getBlockPoolId());
if(bpos != null) {
bpos.notifyNamenodeReceivedBlock(block, delHint);
bpos.notifyNamenodeReceivedBlock(block, delHint, storageUuid);
} else {
LOG.warn("Cannot find BPOfferService for reporting block received for bpid="
+ block.getBlockPoolId());
@ -1675,17 +1710,32 @@ public static DataNode instantiateDataNode(String args [], Configuration conf,
printUsage(System.err);
return null;
}
Collection<URI> dataDirs = getStorageDirs(conf);
Collection<StorageLocation> dataLocations = getStorageLocations(conf);
UserGroupInformation.setConfiguration(conf);
SecurityUtil.login(conf, DFS_DATANODE_KEYTAB_FILE_KEY,
DFS_DATANODE_USER_NAME_KEY);
return makeInstance(dataDirs, conf, resources);
return makeInstance(dataLocations, conf, resources);
}
static Collection<URI> getStorageDirs(Configuration conf) {
Collection<String> dirNames =
conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
return Util.stringCollectionAsURIs(dirNames);
public static List<StorageLocation> getStorageLocations(Configuration conf) {
Collection<String> rawLocations =
conf.getTrimmedStringCollection(DFS_DATANODE_DATA_DIR_KEY);
List<StorageLocation> locations =
new ArrayList<StorageLocation>(rawLocations.size());
for(String locationString : rawLocations) {
final StorageLocation location;
try {
location = StorageLocation.parse(locationString);
} catch (IOException ioe) {
throw new IllegalArgumentException("Failed to parse conf property "
+ DFS_DATANODE_DATA_DIR_KEY + ": " + locationString, ioe);
}
locations.add(location);
}
return locations;
}
/** Instantiate & Start a single datanode daemon and wait for it to finish.
@ -1751,57 +1801,52 @@ public void checkDir(LocalFileSystem localFS, Path path)
* no directory from this directory list can be created.
* @throws IOException
*/
static DataNode makeInstance(Collection<URI> dataDirs, Configuration conf,
SecureResources resources) throws IOException {
static DataNode makeInstance(Collection<StorageLocation> dataDirs,
Configuration conf, SecureResources resources) throws IOException {
LocalFileSystem localFS = FileSystem.getLocal(conf);
FsPermission permission = new FsPermission(
conf.get(DFS_DATANODE_DATA_DIR_PERMISSION_KEY,
DFS_DATANODE_DATA_DIR_PERMISSION_DEFAULT));
DataNodeDiskChecker dataNodeDiskChecker =
new DataNodeDiskChecker(permission);
ArrayList<File> dirs =
getDataDirsFromURIs(dataDirs, localFS, dataNodeDiskChecker);
List<StorageLocation> locations =
checkStorageLocations(dataDirs, localFS, dataNodeDiskChecker);
DefaultMetricsSystem.initialize("DataNode");
assert dirs.size() > 0 : "number of data directories should be > 0";
return new DataNode(conf, dirs, resources);
assert locations.size() > 0 : "number of data directories should be > 0";
return new DataNode(conf, locations, resources);
}
// DataNode ctor expects AbstractList instead of List or Collection...
static ArrayList<File> getDataDirsFromURIs(Collection<URI> dataDirs,
static List<StorageLocation> checkStorageLocations(
Collection<StorageLocation> dataDirs,
LocalFileSystem localFS, DataNodeDiskChecker dataNodeDiskChecker)
throws IOException {
ArrayList<File> dirs = new ArrayList<File>();
ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
StringBuilder invalidDirs = new StringBuilder();
for (URI dirURI : dataDirs) {
if (!"file".equalsIgnoreCase(dirURI.getScheme())) {
LOG.warn("Unsupported URI schema in " + dirURI + ". Ignoring ...");
invalidDirs.append("\"").append(dirURI).append("\" ");
continue;
}
// drop any (illegal) authority in the URI for backwards compatibility
File dir = new File(dirURI.getPath());
for (StorageLocation location : dataDirs) {
final URI uri = location.getUri();
try {
dataNodeDiskChecker.checkDir(localFS, new Path(dir.toURI()));
dirs.add(dir);
dataNodeDiskChecker.checkDir(localFS, new Path(uri));
locations.add(location);
} catch (IOException ioe) {
LOG.warn("Invalid " + DFS_DATANODE_DATA_DIR_KEY + " "
+ dir + " : ", ioe);
invalidDirs.append("\"").append(dirURI.getPath()).append("\" ");
+ location.getFile() + " : ", ioe);
invalidDirs.append("\"").append(uri.getPath()).append("\" ");
}
}
if (dirs.size() == 0) {
if (locations.size() == 0) {
throw new IOException("All directories in "
+ DFS_DATANODE_DATA_DIR_KEY + " are invalid: "
+ invalidDirs);
}
return dirs;
return locations;
}
@Override
public String toString() {
return "DataNode{data=" + data + ", localName='" + getDisplayName()
+ "', storageID='" + getStorageId() + "', xmitsInProgress="
+ "', datanodeUuid='" + storage.getDatanodeUuid() + "', xmitsInProgress="
+ xmitsInProgress.get() + "}";
}
@ -1855,7 +1900,6 @@ public void scheduleAllBlockReport(long delay) {
}
/**
* This method is used for testing.
* Examples are adding and deleting blocks directly.
* The most common usage will be when the data node's storage is simulated.
*
@ -1955,7 +1999,7 @@ public String updateReplicaUnderRecovery(final ExtendedBlock oldBlock,
ExtendedBlock newBlock = new ExtendedBlock(oldBlock);
newBlock.setGenerationStamp(recoveryId);
newBlock.setNumBytes(newLength);
notifyNamenodeReceivedBlock(newBlock, "");
notifyNamenodeReceivedBlock(newBlock, "", storageID);
return storageID;
}
@ -2415,6 +2459,10 @@ public DNConf getDnConf() {
return dnConf;
}
public String getDatanodeUuid() {
return id == null ? null : id.getDatanodeUuid();
}
boolean shouldRun() {
return shouldRun;
}

View File

@ -24,13 +24,7 @@
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.channels.FileLock;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Properties;
import java.util.*;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
@ -50,6 +44,7 @@
import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.util.Daemon;
@ -71,8 +66,13 @@ public class DataStorage extends Storage {
public final static String STORAGE_DIR_FINALIZED = "finalized";
public final static String STORAGE_DIR_TMP = "tmp";
/** Unique storage ID. {@see DataNode#createNewStorageId(int)} for details */
private String storageID;
/**
* Datanode UUID that this storage is currently attached to. This
* is the same as the legacy StorageID for datanodes that were
* upgraded from a pre-UUID version. For compatibility with prior
* versions of Datanodes we cannot make this field a UUID.
*/
private String datanodeUuid = null;
// Flag to ensure we only initialize storage once
private boolean initialized = false;
@ -84,33 +84,29 @@ public class DataStorage extends Storage {
DataStorage() {
super(NodeType.DATA_NODE);
storageID = "";
}
public StorageInfo getBPStorage(String bpid) {
return bpStorageMap.get(bpid);
}
public DataStorage(StorageInfo storageInfo, String strgID) {
public DataStorage(StorageInfo storageInfo) {
super(NodeType.DATA_NODE, storageInfo);
this.storageID = strgID;
}
/** @return storage ID. */
public synchronized String getStorageID() {
return storageID;
public synchronized String getDatanodeUuid() {
return datanodeUuid;
}
synchronized void setStorageID(String newStorageID) {
this.storageID = newStorageID;
public synchronized void setDatanodeUuid(String newDatanodeUuid) {
this.datanodeUuid = newDatanodeUuid;
}
/** Create an ID for this storage. */
public synchronized void createStorageID(int datanodePort) {
if (storageID != null && !storageID.isEmpty()) {
return;
public synchronized void createStorageID(StorageDirectory sd) {
if (sd.getStorageUuid() == null) {
sd.setStorageUuid(DatanodeStorage.generateUuid());
}
storageID = DataNode.createNewStorageId(datanodePort);
}
/**
@ -128,7 +124,8 @@ public synchronized void createStorageID(int datanodePort) {
* @throws IOException
*/
synchronized void recoverTransitionRead(DataNode datanode,
NamespaceInfo nsInfo, Collection<File> dataDirs, StartupOption startOpt)
NamespaceInfo nsInfo, Collection<StorageLocation> dataDirs,
StartupOption startOpt)
throws IOException {
if (initialized) {
// DN storage has been initialized, no need to do anything
@ -144,8 +141,8 @@ synchronized void recoverTransitionRead(DataNode datanode,
// Format and recover.
this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
ArrayList<StorageState> dataDirStates = new ArrayList<StorageState>(dataDirs.size());
for(Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
File dataDir = it.next();
for(Iterator<StorageLocation> it = dataDirs.iterator(); it.hasNext();) {
File dataDir = it.next().getFile();
StorageDirectory sd = new StorageDirectory(dataDir);
StorageState curState;
try {
@ -162,7 +159,7 @@ synchronized void recoverTransitionRead(DataNode datanode,
case NOT_FORMATTED: // format
LOG.info("Storage directory " + dataDir + " is not formatted");
LOG.info("Formatting ...");
format(sd, nsInfo);
format(sd, nsInfo, datanode.getDatanodeUuid());
break;
default: // recovery part is common
sd.doRecover(curState);
@ -191,11 +188,9 @@ synchronized void recoverTransitionRead(DataNode datanode,
doTransition(datanode, getStorageDir(idx), nsInfo, startOpt);
assert this.getLayoutVersion() == nsInfo.getLayoutVersion() :
"Data-node and name-node layout versions must be the same.";
createStorageID(getStorageDir(idx));
}
// make sure we have storage id set - if not - generate new one
createStorageID(datanode.getXferPort());
// 3. Update all storages. Some of them might have just been formatted.
this.writeAll();
@ -214,14 +209,14 @@ synchronized void recoverTransitionRead(DataNode datanode,
* @throws IOException on error
*/
void recoverTransitionRead(DataNode datanode, String bpID, NamespaceInfo nsInfo,
Collection<File> dataDirs, StartupOption startOpt) throws IOException {
Collection<StorageLocation> dataDirs, StartupOption startOpt) throws IOException {
// First ensure datanode level format/snapshot/rollback is completed
recoverTransitionRead(datanode, nsInfo, dataDirs, startOpt);
// Create list of storage directories for the block pool
Collection<File> bpDataDirs = new ArrayList<File>();
for(Iterator<File> it = dataDirs.iterator(); it.hasNext();) {
File dnRoot = it.next();
for(StorageLocation dir : dataDirs) {
File dnRoot = dir.getFile();
File bpRoot = BlockPoolSliceStorage.getBpRoot(bpID, new File(dnRoot,
STORAGE_DIR_CURRENT));
bpDataDirs.add(bpRoot);
@ -263,19 +258,28 @@ static void makeBlockPoolDataDir(Collection<File> dataDirs,
}
}
void format(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
void format(StorageDirectory sd, NamespaceInfo nsInfo,
String datanodeUuid) throws IOException {
sd.clearDirectory(); // create directory
this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
this.clusterID = nsInfo.getClusterID();
this.namespaceID = nsInfo.getNamespaceID();
this.cTime = 0;
// store storageID as it currently is
this.datanodeUuid = datanodeUuid;
if (sd.getStorageUuid() == null) {
// Assign a new Storage UUID.
sd.setStorageUuid(DatanodeStorage.generateUuid());
}
writeProperties(sd);
}
/*
* Set ClusterID, StorageID, StorageType, CTime into
* DataStorage VERSION file
* DataStorage VERSION file.
* Always called just before writing the properties to
* the VERSION file.
*/
@Override
protected void setPropertiesFromFields(Properties props,
@ -285,7 +289,13 @@ protected void setPropertiesFromFields(Properties props,
props.setProperty("clusterID", clusterID);
props.setProperty("cTime", String.valueOf(cTime));
props.setProperty("layoutVersion", String.valueOf(layoutVersion));
props.setProperty("storageID", getStorageID());
props.setProperty("storageID", sd.getStorageUuid());
String datanodeUuid = getDatanodeUuid();
if (datanodeUuid != null) {
props.setProperty("datanodeUuid", datanodeUuid);
}
// Set NamespaceID in version before federation
if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
props.setProperty("namespaceID", String.valueOf(namespaceID));
@ -295,6 +305,7 @@ protected void setPropertiesFromFields(Properties props,
/*
* Read ClusterID, StorageID, StorageType, CTime from
* DataStorage VERSION file and verify them.
* Always called just after reading the properties from the VERSION file.
*/
@Override
protected void setFieldsFromProperties(Properties props, StorageDirectory sd)
@ -318,20 +329,36 @@ private void setFieldsFromProperties(Properties props, StorageDirectory sd,
setNamespaceID(props, sd);
}
// valid storage id, storage id may be empty
String ssid = props.getProperty("storageID");
if (ssid == null) {
throw new InconsistentFSStateException(sd.getRoot(), "file "
+ STORAGE_FILE_VERSION + " is invalid.");
}
String sid = getStorageID();
if (!(sid.equals("") || ssid.equals("") || sid.equals(ssid))) {
String sid = sd.getStorageUuid();
if (!(sid == null || sid.equals("") ||
ssid.equals("") || sid.equals(ssid))) {
throw new InconsistentFSStateException(sd.getRoot(),
"has incompatible storage Id.");
}
if (sid.equals("")) { // update id only if it was empty
setStorageID(ssid);
if (sid == null) { // update id only if it was null
sd.setStorageUuid(ssid);
}
// Update the datanode UUID if present.
if (props.getProperty("datanodeUuid") != null) {
String dnUuid = props.getProperty("datanodeUuid");
if (getDatanodeUuid() == null) {
setDatanodeUuid(dnUuid);
} else if (getDatanodeUuid().compareTo(dnUuid) != 0) {
throw new InconsistentFSStateException(sd.getRoot(),
"Root " + sd.getRoot() + ": DatanodeUuid=" + dnUuid +
", does not match " + getDatanodeUuid() + " from other" +
" StorageDirectory.");
}
}
}

View File

@ -284,7 +284,7 @@ public void requestShortCircuitFds(final ExtendedBlock blk,
BlockSender.ClientTraceLog.info(String.format(
"src: 127.0.0.1, dest: 127.0.0.1, op: REQUEST_SHORT_CIRCUIT_FDS," +
" blockid: %s, srvID: %s, success: %b",
blk.getBlockId(), dnR.getStorageID(), (fis != null)
blk.getBlockId(), dnR.getDatanodeUuid(), (fis != null)
));
}
if (fis != null) {
@ -317,7 +317,7 @@ public void readBlock(final ExtendedBlock block,
clientName.length() > 0 && ClientTraceLog.isInfoEnabled()
? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress,
"%d", "HDFS_READ", clientName, "%d",
dnR.getStorageID(), block, "%d")
dnR.getDatanodeUuid(), block, "%d")
: dnR + " Served block " + block + " to " +
remoteAddress;
@ -447,6 +447,7 @@ public void writeBlock(final ExtendedBlock block,
String mirrorNode = null; // the name:port of next target
String firstBadLink = ""; // first datanode that failed in connection setup
Status mirrorInStatus = SUCCESS;
final String storageUuid;
try {
if (isDatanode ||
stage != BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
@ -457,8 +458,10 @@ public void writeBlock(final ExtendedBlock block,
stage, latestGenerationStamp, minBytesRcvd, maxBytesRcvd,
clientname, srcDataNode, datanode, requestedChecksum,
cachingStrategy);
storageUuid = blockReceiver.getStorageUuid();
} else {
datanode.data.recoverClose(block, latestGenerationStamp, minBytesRcvd);
storageUuid = datanode.data.recoverClose(
block, latestGenerationStamp, minBytesRcvd);
}
//
@ -590,7 +593,7 @@ public void writeBlock(final ExtendedBlock block,
// the block is finalized in the PacketResponder.
if (isDatanode ||
stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT);
datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT, storageUuid);
LOG.info("Received " + block + " src: " + remoteAddress + " dest: "
+ localAddress + " of size " + block.getNumBytes());
}
@ -859,9 +862,11 @@ public void replaceBlock(final ExtendedBlock block,
dataXceiverServer.balanceThrottler, null);
// notify name node
datanode.notifyNamenodeReceivedBlock(block, delHint);
datanode.notifyNamenodeReceivedBlock(
block, delHint, blockReceiver.getStorageUuid());
LOG.info("Moved " + block + " from " + peer.getRemoteAddressString());
LOG.info("Moved " + block + " from " + peer.getRemoteAddressString()
+ ", delHint=" + delHint);
} catch (IOException ioe) {
opStatus = ERROR;

View File

@ -19,7 +19,6 @@
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedList;
@ -230,10 +229,6 @@ private static String getSuffix(File f, String prefix) {
throw new RuntimeException(prefix + " is not a prefix of " + fullPath);
}
ScanInfo(long blockId) {
this(blockId, null, null, null);
}
ScanInfo(long blockId, File blockFile, File metaFile, FsVolumeSpi vol) {
this.blockId = blockId;
String condensedVolPath = vol == null ? null :
@ -439,8 +434,8 @@ void scan() {
diffs.put(bpid, diffRecord);
statsRecord.totalBlocks = blockpoolReport.length;
List<Block> bl = dataset.getFinalizedBlocks(bpid);
Block[] memReport = bl.toArray(new Block[bl.size()]);
List<FinalizedReplica> bl = dataset.getFinalizedBlocks(bpid);
FinalizedReplica[] memReport = bl.toArray(new FinalizedReplica[bl.size()]);
Arrays.sort(memReport); // Sort based on blockId
int d = 0; // index for blockpoolReport
@ -458,7 +453,8 @@ void scan() {
}
if (info.getBlockId() > memBlock.getBlockId()) {
// Block is missing on the disk
addDifference(diffRecord, statsRecord, memBlock.getBlockId());
addDifference(diffRecord, statsRecord,
memBlock.getBlockId(), info.getVolume());
m++;
continue;
}
@ -478,7 +474,9 @@ void scan() {
m++;
}
while (m < memReport.length) {
addDifference(diffRecord, statsRecord, memReport[m++].getBlockId());
FinalizedReplica current = memReport[m++];
addDifference(diffRecord, statsRecord,
current.getBlockId(), current.getVolume());
}
while (d < blockpoolReport.length) {
statsRecord.missingMemoryBlocks++;
@ -502,10 +500,11 @@ private void addDifference(LinkedList<ScanInfo> diffRecord,
/** Block is not found on the disk */
private void addDifference(LinkedList<ScanInfo> diffRecord,
Stats statsRecord, long blockId) {
Stats statsRecord, long blockId,
FsVolumeSpi vol) {
statsRecord.missingBlockFile++;
statsRecord.missingMetaFile++;
diffRecord.add(new ScanInfo(blockId));
diffRecord.add(new ScanInfo(blockId, null, null, vol));
}
/** Is the given volume still valid in the dataset? */

View File

@ -54,4 +54,9 @@ public interface Replica {
* @return the number of bytes that are visible to readers
*/
public long getVisibleLength();
/**
* Return the storageUuid of the volume that stores this replica.
*/
public String getStorageUuid();
}

View File

@ -137,6 +137,14 @@ public FsVolumeSpi getVolume() {
void setVolume(FsVolumeSpi vol) {
this.volume = vol;
}
/**
* Get the storageUuid of the volume that stores this replica.
*/
@Override
public String getStorageUuid() {
return volume.getStorageID();
}
/**
* Return the parent directory path where this replica is located

View File

@ -0,0 +1,101 @@
/**
* 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.datanode;
import java.util.regex.Pattern;
import java.io.File;
import java.io.IOException;
import java.net.URI;
import java.util.regex.Matcher;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.server.common.Util;
/**
* Encapsulates the URI and storage medium that together describe a
* storage directory.
* The default storage medium is assumed to be DISK, if none is specified.
*
*/
@InterfaceAudience.Private
public class StorageLocation {
final StorageType storageType;
final File file;
/** Regular expression that describes a storage uri with a storage type.
* e.g. [Disk]/storages/storage1/
*/
private static final Pattern regex = Pattern.compile("^\\[(\\w*)\\](.+)$");
private StorageLocation(StorageType storageType, URI uri) {
this.storageType = storageType;
if (uri.getScheme() == null ||
"file".equalsIgnoreCase(uri.getScheme())) {
// drop any (illegal) authority in the URI for backwards compatibility
this.file = new File(uri.getPath());
} else {
throw new IllegalArgumentException("Unsupported URI schema in " + uri);
}
}
public StorageType getStorageType() {
return this.storageType;
}
URI getUri() {
return file.toURI();
}
public File getFile() {
return this.file;
}
/**
* Attempt to parse a storage uri with storage class and URI. The storage
* class component of the uri is case-insensitive.
*
* @param rawLocation Location string of the format [type]uri, where [type] is
* optional.
* @return A StorageLocation object if successfully parsed, null otherwise.
* Does not throw any exceptions.
*/
static StorageLocation parse(String rawLocation) throws IOException {
Matcher matcher = regex.matcher(rawLocation);
StorageType storageType = StorageType.DEFAULT;
String location = rawLocation;
if (matcher.matches()) {
String classString = matcher.group(1);
location = matcher.group(2);
if (!classString.isEmpty()) {
storageType = StorageType.valueOf(classString.toUpperCase());
}
}
return new StorageLocation(storageType, Util.stringAsURI(location));
}
@Override
public String toString() {
return "[" + storageType + "]" + file.toURI();
}
}

View File

@ -34,12 +34,15 @@
import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
import org.apache.hadoop.hdfs.server.datanode.Replica;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory;
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
import org.apache.hadoop.util.ReflectionUtils;
@ -86,17 +89,18 @@ public RollingLogs createRollingLogs(String bpid, String prefix
/** @return a list of volumes. */
public List<V> getVolumes();
/** @return one or more storage reports for attached volumes. */
public StorageReport[] getStorageReports(String bpid)
throws IOException;
/** @return the volume that contains a replica of the block. */
public V getVolume(ExtendedBlock b);
/** @return a volume information map (name => info). */
public Map<String, Object> getVolumeInfoMap();
/** @return a list of block pools. */
public String[] getBlockPoolList();
/** @return a list of finalized blocks for the given block pool. */
public List<Block> getFinalizedBlocks(String bpid);
public List<FinalizedReplica> getFinalizedBlocks(String bpid);
/**
* Check whether the in-memory block record matches the block on the disk,
@ -239,9 +243,10 @@ public ReplicaInPipelineInterface recoverAppend(ExtendedBlock b, long newGS,
* @param b block
* @param newGS the new generation stamp for the replica
* @param expectedBlockLen the number of bytes the replica is expected to have
* @return the storage uuid of the replica.
* @throws IOException
*/
public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
) throws IOException;
/**
@ -262,12 +267,11 @@ public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen
public void unfinalizeBlock(ExtendedBlock b) throws IOException;
/**
* Returns the block report - the full list of blocks stored under a
* block pool
* Returns one block report per volume.
* @param bpid Block Pool Id
* @return - the block report - the full list of blocks stored
* @return - a map of DatanodeStorage to block report for the volume.
*/
public BlockListAsLongs getBlockReport(String bpid);
public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid);
/** Does the dataset contain the block? */
public boolean contains(ExtendedBlock block);
@ -386,3 +390,4 @@ public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock b
public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
throws IOException;
}

View File

@ -20,10 +20,15 @@
import java.io.File;
import java.io.IOException;
import org.apache.hadoop.hdfs.StorageType;
/**
* This is an interface for the underlying volume.
*/
public interface FsVolumeSpi {
/** @return the StorageUuid of the volume */
public String getStorageID();
/** @return a list of block pools. */
public String[] getBlockPoolList();
@ -38,4 +43,6 @@ public interface FsVolumeSpi {
/** @return the directory for the finalized blocks in the block pool. */
public File getFinalizedDir(String bpid) throws IOException;
public StorageType getStorageType();
}

View File

@ -195,7 +195,7 @@ public void run() {
+ " at file " + blockFile + ". Ignored.");
} else {
if(block.getLocalBlock().getNumBytes() != BlockCommand.NO_ACK){
datanode.notifyNamenodeDeletedBlock(block);
datanode.notifyNamenodeDeletedBlock(block, volume.getStorageID());
}
volume.decDfsUsed(block.getBlockPoolId(), dfsBytes);
LOG.info("Deleted " + block.getBlockPoolId() + " "

View File

@ -37,13 +37,14 @@
import javax.management.ObjectName;
import javax.management.StandardMBean;
import com.google.common.base.Preconditions;
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.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
@ -52,6 +53,7 @@
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
import org.apache.hadoop.hdfs.server.datanode.DataBlockScanner;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
@ -65,6 +67,7 @@
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
import org.apache.hadoop.hdfs.server.datanode.ReplicaUnderRecovery;
import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.LengthInputStream;
@ -75,7 +78,9 @@
import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.io.nativeio.NativeIO;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.util.DataChecksum;
@ -107,6 +112,26 @@ public List<FsVolumeImpl> getVolumes() {
return volumes.volumes;
}
@Override // FsDatasetSpi
public StorageReport[] getStorageReports(String bpid)
throws IOException {
StorageReport[] reports;
synchronized (statsLock) {
reports = new StorageReport[volumes.volumes.size()];
int i = 0;
for (FsVolumeImpl volume : volumes.volumes) {
reports[i++] = new StorageReport(volume.getStorageID(),
false,
volume.getCapacity(),
volume.getDfsUsed(),
volume.getAvailable(),
volume.getBlockPoolUsed(bpid));
}
}
return reports;
}
@Override
public synchronized FsVolumeImpl getVolume(final ExtendedBlock b) {
final ReplicaInfo r = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock());
@ -168,10 +193,11 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b)
final DataNode datanode;
final FsVolumeList volumes;
final ReplicaMap volumeMap;
final FsDatasetAsyncDiskService asyncDiskService;
private final int validVolsRequired;
final ReplicaMap volumeMap;
// Used for synchronizing access to usage stats
private final Object statsLock = new Object();
@ -188,6 +214,7 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b)
DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
String[] dataDirs = conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
Collection<StorageLocation> dataLocations = DataNode.getStorageLocations(conf);
int volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
int volsFailed = volsConfigured - storage.getNumStorageDirs();
@ -208,9 +235,15 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b)
final List<FsVolumeImpl> volArray = new ArrayList<FsVolumeImpl>(
storage.getNumStorageDirs());
for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
final File dir = storage.getStorageDir(idx).getCurrentDir();
volArray.add(new FsVolumeImpl(this, storage.getStorageID(), dir, conf));
LOG.info("Added volume - " + dir);
// TODO: getStorageTypeFromLocations() is only a temporary workaround and
// should be replaced with getting storage type from DataStorage (missing
// storage type now) directly.
Storage.StorageDirectory sd = storage.getStorageDir(idx);
final File dir = sd.getCurrentDir();
final StorageType storageType = getStorageTypeFromLocations(dataLocations, dir);
volArray.add(new FsVolumeImpl(this, sd.getStorageUuid(), dir, conf,
storageType));
LOG.info("Added volume - " + dir + ", StorageType: " + storageType);
}
volumeMap = new ReplicaMap(this);
@ -221,14 +254,24 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b)
RoundRobinVolumeChoosingPolicy.class,
VolumeChoosingPolicy.class), conf);
volumes = new FsVolumeList(volArray, volsFailed, blockChooserImpl);
volumes.getVolumeMap(volumeMap);
volumes.initializeReplicaMaps(volumeMap);
File[] roots = new File[storage.getNumStorageDirs()];
for (int idx = 0; idx < storage.getNumStorageDirs(); idx++) {
roots[idx] = storage.getStorageDir(idx).getCurrentDir();
}
asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots);
registerMBean(storage.getStorageID());
registerMBean(datanode.getDatanodeUuid());
}
private StorageType getStorageTypeFromLocations(
Collection<StorageLocation> dataLocations, File dir) {
for (StorageLocation dataLocation : dataLocations) {
if (dataLocation.getFile().equals(dir)) {
return dataLocation.getStorageType();
}
}
return StorageType.DEFAULT;
}
/**
@ -308,9 +351,6 @@ private File getBlockFile(ExtendedBlock b) throws IOException {
File getBlockFile(String bpid, Block b) throws IOException {
File f = validateBlockFile(bpid, b);
if(f == null) {
if (LOG.isDebugEnabled()) {
LOG.debug("b=" + b + ", volumeMap=" + volumeMap);
}
throw new IOException("Block " + b + " is not valid.");
}
return f;
@ -654,7 +694,7 @@ public synchronized ReplicaInPipeline recoverAppend(ExtendedBlock b,
}
@Override // FsDatasetSpi
public void recoverClose(ExtendedBlock b, long newGS,
public String recoverClose(ExtendedBlock b, long newGS,
long expectedBlockLen) throws IOException {
LOG.info("Recover failed close " + b);
// check replica's state
@ -665,6 +705,7 @@ public void recoverClose(ExtendedBlock b, long newGS,
if (replicaInfo.getState() == ReplicaState.RBW) {
finalizeReplica(b.getBlockPoolId(), replicaInfo);
}
return replicaInfo.getStorageUuid();
}
/**
@ -965,51 +1006,63 @@ private boolean delBlockFromDisk(File blockFile, File metaFile, Block b) {
return true;
}
/**
* Generates a block report from the in-memory block map.
*/
@Override // FsDatasetSpi
public BlockListAsLongs getBlockReport(String bpid) {
int size = volumeMap.size(bpid);
ArrayList<ReplicaInfo> finalized = new ArrayList<ReplicaInfo>(size);
ArrayList<ReplicaInfo> uc = new ArrayList<ReplicaInfo>();
if (size == 0) {
return new BlockListAsLongs(finalized, uc);
@Override
public Map<DatanodeStorage, BlockListAsLongs> getBlockReports(String bpid) {
Map<DatanodeStorage, BlockListAsLongs> blockReportsMap =
new HashMap<DatanodeStorage, BlockListAsLongs>();
Map<String, ArrayList<ReplicaInfo>> finalized =
new HashMap<String, ArrayList<ReplicaInfo>>();
Map<String, ArrayList<ReplicaInfo>> uc =
new HashMap<String, ArrayList<ReplicaInfo>>();
for (FsVolumeSpi v : volumes.volumes) {
finalized.put(v.getStorageID(), new ArrayList<ReplicaInfo>());
uc.put(v.getStorageID(), new ArrayList<ReplicaInfo>());
}
synchronized(this) {
for (ReplicaInfo b : volumeMap.replicas(bpid)) {
switch(b.getState()) {
case FINALIZED:
finalized.add(b);
break;
case RBW:
case RWR:
uc.add(b);
break;
case RUR:
ReplicaUnderRecovery rur = (ReplicaUnderRecovery)b;
uc.add(rur.getOriginalReplica());
break;
case TEMPORARY:
break;
default:
assert false : "Illegal ReplicaInfo state.";
case FINALIZED:
finalized.get(b.getVolume().getStorageID()).add(b);
break;
case RBW:
case RWR:
uc.get(b.getVolume().getStorageID()).add(b);
break;
case RUR:
ReplicaUnderRecovery rur = (ReplicaUnderRecovery)b;
uc.get(rur.getVolume().getStorageID()).add(rur.getOriginalReplica());
break;
case TEMPORARY:
break;
default:
assert false : "Illegal ReplicaInfo state.";
}
}
return new BlockListAsLongs(finalized, uc);
}
for (FsVolumeSpi v : volumes.volumes) {
ArrayList<ReplicaInfo> finalizedList = finalized.get(v.getStorageID());
ArrayList<ReplicaInfo> ucList = uc.get(v.getStorageID());
blockReportsMap.put(((FsVolumeImpl) v).toDatanodeStorage(),
new BlockListAsLongs(finalizedList, ucList));
}
return blockReportsMap;
}
/**
* Get the list of finalized blocks from in-memory blockmap for a block pool.
*/
@Override
public synchronized List<Block> getFinalizedBlocks(String bpid) {
ArrayList<Block> finalized = new ArrayList<Block>(volumeMap.size(bpid));
public synchronized List<FinalizedReplica> getFinalizedBlocks(String bpid) {
ArrayList<FinalizedReplica> finalized =
new ArrayList<FinalizedReplica>(volumeMap.size(bpid));
for (ReplicaInfo b : volumeMap.replicas(bpid)) {
if(b.getState() == ReplicaState.FINALIZED) {
finalized.add(new Block(b));
finalized.add(new FinalizedReplica((FinalizedReplica)b));
}
}
return finalized;
@ -1230,22 +1283,15 @@ public String toString() {
/**
* Register the FSDataset MBean using the name
* "hadoop:service=DataNode,name=FSDatasetState-<storageid>"
* "hadoop:service=DataNode,name=FSDatasetState-<datanodeUuid>"
*/
void registerMBean(final String storageId) {
void registerMBean(final String datanodeUuid) {
// We wrap to bypass standard mbean naming convetion.
// This wraping can be removed in java 6 as it is more flexible in
// package naming for mbeans and their impl.
StandardMBean bean;
String storageName;
if (storageId == null || storageId.equals("")) {// Temp fix for the uninitialized storage
storageName = "UndefinedStorageId" + DFSUtil.getRandom().nextInt();
} else {
storageName = storageId;
}
try {
bean = new StandardMBean(this,FSDatasetMBean.class);
mbeanName = MBeans.register("DataNode", "FSDatasetState-" + storageName, bean);
StandardMBean bean = new StandardMBean(this,FSDatasetMBean.class);
mbeanName = MBeans.register("DataNode", "FSDatasetState-" + datanodeUuid, bean);
} catch (NotCompliantMBeanException e) {
LOG.warn("Error registering FSDatasetState MBean", e);
}
@ -1621,7 +1667,7 @@ public synchronized void addBlockPool(String bpid, Configuration conf)
LOG.info("Adding block pool " + bpid);
volumes.addBlockPool(bpid, conf);
volumeMap.initBlockPool(bpid);
volumes.getVolumeMap(bpid, volumeMap);
volumes.getAllVolumesMap(bpid, volumeMap);
}
@Override
@ -1631,11 +1677,6 @@ public synchronized void shutdownBlockPool(String bpid) {
volumes.removeBlockPool(bpid);
}
@Override
public String[] getBlockPoolList() {
return volumeMap.getBlockPoolList();
}
/**
* Class for representing the Datanode volume information
*/
@ -1768,3 +1809,4 @@ public RollingLogs createRollingLogs(String bpid, String prefix
return new RollingLogsImpl(dir, prefix);
}
}

View File

@ -29,9 +29,11 @@
import org.apache.hadoop.fs.DF;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.datanode.DataStorage;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
/**
@ -43,6 +45,7 @@
class FsVolumeImpl implements FsVolumeSpi {
private final FsDatasetImpl dataset;
private final String storageID;
private final StorageType storageType;
private final Map<String, BlockPoolSlice> bpSlices
= new HashMap<String, BlockPoolSlice>();
private final File currentDir; // <StorageDirectory>/current
@ -50,7 +53,7 @@ class FsVolumeImpl implements FsVolumeSpi {
private final long reserved;
FsVolumeImpl(FsDatasetImpl dataset, String storageID, File currentDir,
Configuration conf) throws IOException {
Configuration conf, StorageType storageType) throws IOException {
this.dataset = dataset;
this.storageID = storageID;
this.reserved = conf.getLong(
@ -59,6 +62,7 @@ class FsVolumeImpl implements FsVolumeSpi {
this.currentDir = currentDir;
File parent = currentDir.getParentFile();
this.usage = new DF(parent, conf);
this.storageType = storageType;
}
File getCurrentDir() {
@ -287,7 +291,19 @@ void deleteBPDirectories(String bpid, boolean force) throws IOException {
}
}
String getStorageID() {
@Override
public String getStorageID() {
return storageID;
}
@Override
public StorageType getStorageType() {
return storageType;
}
DatanodeStorage toDatanodeStorage() {
return new DatanodeStorage(storageID, DatanodeStorage.State.NORMAL, storageType);
}
}

View File

@ -18,10 +18,7 @@
package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.*;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
@ -56,6 +53,7 @@ int numberOfFailedVolumes() {
* @param blockSize free space needed on the volume
* @return next volume to store the block in.
*/
// TODO should choose volume with storage type
synchronized FsVolumeImpl getNextVolume(long blockSize) throws IOException {
return blockChooser.chooseVolume(volumes, blockSize);
}
@ -92,27 +90,32 @@ long getRemaining() throws IOException {
return remaining;
}
void getVolumeMap(ReplicaMap volumeMap) throws IOException {
void initializeReplicaMaps(ReplicaMap globalReplicaMap) throws IOException {
for (FsVolumeImpl v : volumes) {
v.getVolumeMap(volumeMap);
v.getVolumeMap(globalReplicaMap);
}
}
void getVolumeMap(String bpid, ReplicaMap volumeMap) throws IOException {
void getAllVolumesMap(String bpid, ReplicaMap volumeMap) throws IOException {
long totalStartTime = System.currentTimeMillis();
for (FsVolumeImpl v : volumes) {
FsDatasetImpl.LOG.info("Adding replicas to map for block pool " + bpid +
" on volume " + v + "...");
long startTime = System.currentTimeMillis();
v.getVolumeMap(bpid, volumeMap);
long timeTaken = System.currentTimeMillis() - startTime;
FsDatasetImpl.LOG.info("Time to add replicas to map for block pool " + bpid +
" on volume " + v + ": " + timeTaken + "ms");
getVolumeMap(bpid, v, volumeMap);
}
long totalTimeTaken = System.currentTimeMillis() - totalStartTime;
FsDatasetImpl.LOG.info("Total time to add all replicas to map: "
+ totalTimeTaken + "ms");
}
void getVolumeMap(String bpid, FsVolumeImpl volume, ReplicaMap volumeMap)
throws IOException {
FsDatasetImpl.LOG.info("Adding replicas to map for block pool " + bpid +
" on volume " + volume + "...");
long startTime = System.currentTimeMillis();
volume.getVolumeMap(bpid, volumeMap);
long timeTaken = System.currentTimeMillis() - startTime;
FsDatasetImpl.LOG.info("Time to add replicas to map for block pool " + bpid +
" on volume " + volume + ": " + timeTaken + "ms");
}
/**
* Calls {@link FsVolumeImpl#checkDirs()} on each volume, removing any

View File

@ -117,6 +117,13 @@ ReplicaInfo add(String bpid, ReplicaInfo replicaInfo) {
return m.put(replicaInfo.getBlockId(), replicaInfo);
}
}
/**
* Add all entries from the given replica map into the local replica map.
*/
void addAll(ReplicaMap other) {
map.putAll(other.map);
}
/**
* Remove the replica's meta information from the map that matches

View File

@ -61,6 +61,7 @@
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.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithCount;
@ -360,7 +361,7 @@ INodeFile unprotectedAddFile( long id,
* Add a block to the file. Returns a reference to the added block.
*/
BlockInfo addBlock(String path, INodesInPath inodesInPath, Block block,
DatanodeDescriptor targets[]) throws IOException {
DatanodeStorageInfo[] targets) throws IOException {
waitForReady();
writeLock();

View File

@ -147,6 +147,7 @@
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.HAUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
@ -170,14 +171,7 @@
import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.AccessMode;
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.BlockCollection;
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.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
import org.apache.hadoop.hdfs.server.blockmanagement.OutOfV1GenerationStampsException;
import org.apache.hadoop.hdfs.server.blockmanagement.*;
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
@ -215,7 +209,8 @@
import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.hdfs.util.ChunkedArrayList;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
@ -2547,7 +2542,7 @@ LocatedBlock getAdditionalBlock(String src, long fileId, String clientName,
}
// choose targets for the new block to be allocated.
final DatanodeDescriptor targets[] = getBlockManager().chooseTarget(
final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget(
src, replication, clientNode, excludedNodes, blockSize, favoredNodes);
// Part II.
@ -2675,7 +2670,7 @@ INodesInPath analyzeFileState(String src,
src + ". Returning previously allocated block " + lastBlockInFile);
long offset = pendingFile.computeFileSize();
onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
((BlockInfoUnderConstruction)lastBlockInFile).getExpectedLocations(),
((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
offset);
return iip;
} else {
@ -2693,11 +2688,10 @@ INodesInPath analyzeFileState(String src,
return iip;
}
LocatedBlock makeLocatedBlock(Block blk,
DatanodeInfo[] locs,
LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
long offset) throws IOException {
LocatedBlock lBlk = new LocatedBlock(
getExtendedBlock(blk), locs, offset);
getExtendedBlock(blk), locs, offset, false);
getBlockManager().setBlockToken(
lBlk, BlockTokenSecretManager.AccessMode.WRITE);
return lBlk;
@ -2705,7 +2699,8 @@ LocatedBlock makeLocatedBlock(Block blk,
/** @see NameNode#getAdditionalDatanode(String, ExtendedBlock, DatanodeInfo[], DatanodeInfo[], int, String) */
LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
final DatanodeInfo[] existings, final Set<Node> excludes,
final DatanodeInfo[] existings, final String[] storageIDs,
final Set<Node> excludes,
final int numAdditionalNodes, final String clientName
) throws IOException {
//check if the feature is enabled
@ -2713,7 +2708,7 @@ LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
final DatanodeDescriptor clientnode;
final long preferredblocksize;
final List<DatanodeDescriptor> chosen;
final List<DatanodeStorageInfo> chosen;
checkOperation(OperationCategory.READ);
byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
readLock();
@ -2728,23 +2723,18 @@ LocatedBlock getAdditionalDatanode(String src, final ExtendedBlock blk,
clientnode = file.getClientNode();
preferredblocksize = file.getPreferredBlockSize();
//find datanode descriptors
chosen = new ArrayList<DatanodeDescriptor>();
for(DatanodeInfo d : existings) {
final DatanodeDescriptor descriptor = blockManager.getDatanodeManager(
).getDatanode(d);
if (descriptor != null) {
chosen.add(descriptor);
}
}
//find datanode storages
final DatanodeManager dm = blockManager.getDatanodeManager();
chosen = Arrays.asList(dm.getDatanodeStorageInfos(existings, storageIDs));
} finally {
readUnlock();
}
// choose new datanodes.
final DatanodeInfo[] targets = blockManager.getBlockPlacementPolicy(
final DatanodeStorageInfo[] targets = blockManager.getBlockPlacementPolicy(
).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true,
excludes, preferredblocksize);
// TODO: get storage type from the file
excludes, preferredblocksize, StorageType.DEFAULT);
final LocatedBlock lb = new LocatedBlock(blk, targets);
blockManager.setBlockToken(lb, AccessMode.COPY);
return lb;
@ -2918,14 +2908,13 @@ private boolean completeFileInternal(String src,
* @throws QuotaExceededException If addition of block exceeds space quota
*/
BlockInfo saveAllocatedBlock(String src, INodesInPath inodes,
Block newBlock, DatanodeDescriptor targets[]) throws IOException {
Block newBlock, DatanodeStorageInfo[] targets)
throws IOException {
assert hasWriteLock();
BlockInfo b = dir.addBlock(src, inodes, newBlock, targets);
NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
+ getBlockPoolId() + " " + b);
for (DatanodeDescriptor dn : targets) {
dn.incBlocksScheduled();
}
DatanodeStorageInfo.incrementBlocksScheduled(targets);
return b;
}
@ -3388,7 +3377,7 @@ HdfsFileStatus getFileInfo(String src, boolean resolveLink)
boolean isFileClosed(String src)
throws AccessControlException, UnresolvedLinkException,
StandbyException, IOException {
FSPermissionChecker pc = getPermissionChecker();
FSPermissionChecker pc = getPermissionChecker();
checkOperation(OperationCategory.READ);
readLock();
try {
@ -3685,7 +3674,7 @@ boolean internalReleaseLease(Lease lease, String src,
final BlockInfoUnderConstruction uc = (BlockInfoUnderConstruction)lastBlock;
// setup the last block locations from the blockManager if not known
if (uc.getNumExpectedLocations() == 0) {
uc.setExpectedLocations(blockManager.getNodes(lastBlock));
uc.setExpectedLocations(blockManager.getStorages(lastBlock));
}
if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) {
@ -3883,32 +3872,39 @@ void commitBlockSynchronization(ExtendedBlock lastblock,
// find the DatanodeDescriptor objects
// There should be no locations in the blockManager till now because the
// file is underConstruction
List<DatanodeDescriptor> targetList =
ArrayList<DatanodeDescriptor> trimmedTargets =
new ArrayList<DatanodeDescriptor>(newtargets.length);
ArrayList<String> trimmedStorages =
new ArrayList<String>(newtargets.length);
if (newtargets.length > 0) {
for (DatanodeID newtarget : newtargets) {
for (int i = 0; i < newtargets.length; ++i) {
// try to get targetNode
DatanodeDescriptor targetNode =
blockManager.getDatanodeManager().getDatanode(newtarget);
if (targetNode != null)
targetList.add(targetNode);
else if (LOG.isDebugEnabled()) {
LOG.debug("DatanodeDescriptor (=" + newtarget + ") not found");
blockManager.getDatanodeManager().getDatanode(newtargets[i]);
if (targetNode != null) {
trimmedTargets.add(targetNode);
trimmedStorages.add(newtargetstorages[i]);
} else if (LOG.isDebugEnabled()) {
LOG.debug("DatanodeDescriptor (=" + newtargets[i] + ") not found");
}
}
}
if ((closeFile) && !targetList.isEmpty()) {
if ((closeFile) && !trimmedTargets.isEmpty()) {
// the file is getting closed. Insert block locations into blockManager.
// Otherwise fsck will report these blocks as MISSING, especially if the
// blocksReceived from Datanodes take a long time to arrive.
for (DatanodeDescriptor targetNode : targetList) {
targetNode.addBlock(storedBlock);
for (int i = 0; i < trimmedTargets.size(); i++) {
trimmedTargets.get(i).addBlock(
trimmedStorages.get(i), storedBlock);
}
}
// add pipeline locations into the INodeUnderConstruction
DatanodeDescriptor[] targetArray =
new DatanodeDescriptor[targetList.size()];
pendingFile.setLastBlock(storedBlock, targetList.toArray(targetArray));
DatanodeStorageInfo[] trimmedStorageInfos =
blockManager.getDatanodeManager().getDatanodeStorageInfos(
trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]),
trimmedStorages.toArray(new String[trimmedStorages.size()]));
pendingFile.setLastBlock(storedBlock, trimmedStorageInfos);
}
if (closeFile) {
@ -4112,16 +4108,15 @@ String getRegistrationID() {
* @throws IOException
*/
HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
int xceiverCount, int xmitsInProgress, int failedVolumes)
StorageReport[] reports, int xceiverCount, int xmitsInProgress,
int failedVolumes)
throws IOException {
readLock();
try {
final int maxTransfer = blockManager.getMaxReplicationStreams()
- xmitsInProgress;
DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed,
xceiverCount, maxTransfer, failedVolumes);
nodeReg, reports, blockPoolId, xceiverCount, maxTransfer, failedVolumes);
return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
} finally {
readUnlock();
@ -5283,11 +5278,11 @@ NamenodeCommand startCheckpoint(NamenodeRegistration backupNode,
}
public void processIncrementalBlockReport(final DatanodeID nodeID,
final String poolId, final ReceivedDeletedBlockInfo blockInfos[])
final String poolId, final StorageReceivedDeletedBlocks srdb)
throws IOException {
writeLock();
try {
blockManager.processIncrementalBlockReport(nodeID, poolId, blockInfos);
blockManager.processIncrementalBlockReport(nodeID, poolId, srdb);
} finally {
writeUnlock();
}
@ -5775,9 +5770,10 @@ void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
for (int i = 0; i < blocks.length; i++) {
ExtendedBlock blk = blocks[i].getBlock();
DatanodeInfo[] nodes = blocks[i].getLocations();
String[] storageIDs = blocks[i].getStorageIDs();
for (int j = 0; j < nodes.length; j++) {
DatanodeInfo dn = nodes[j];
blockManager.findAndMarkBlockAsCorrupt(blk, dn,
blockManager.findAndMarkBlockAsCorrupt(blk, nodes[j],
storageIDs == null ? null: storageIDs[j],
"client machine reported it");
}
}
@ -5832,7 +5828,7 @@ LocatedBlock updateBlockForPipeline(ExtendedBlock block,
* @throws IOException if any error occurs
*/
void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes)
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
throws IOException {
checkOperation(OperationCategory.WRITE);
CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
@ -5853,7 +5849,7 @@ void updatePipeline(String clientName, ExtendedBlock oldBlock,
assert newBlock.getBlockId()==oldBlock.getBlockId() : newBlock + " and "
+ oldBlock + " has different block identifier";
updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
cacheEntry != null);
newStorageIDs, cacheEntry != null);
success = true;
} finally {
writeUnlock();
@ -5865,7 +5861,8 @@ void updatePipeline(String clientName, ExtendedBlock oldBlock,
/** @see #updatePipeline(String, ExtendedBlock, ExtendedBlock, DatanodeID[]) */
private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes, boolean logRetryCache)
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs,
boolean logRetryCache)
throws IOException {
assert hasWriteLock();
// check the vadility of the block and lease holder name
@ -5889,15 +5886,9 @@ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock,
blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
// find the DatanodeDescriptor objects
final DatanodeManager dm = getBlockManager().getDatanodeManager();
DatanodeDescriptor[] descriptors = null;
if (newNodes.length > 0) {
descriptors = new DatanodeDescriptor[newNodes.length];
for(int i = 0; i < newNodes.length; i++) {
descriptors[i] = dm.getDatanode(newNodes[i]);
}
}
blockinfo.setExpectedLocations(descriptors);
final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()
.getDatanodeStorageInfos(newNodes, newStorageIDs);
blockinfo.setExpectedLocations(storages);
String src = leaseManager.findPath(pendingFile);
dir.persistBlocks(src, pendingFile, logRetryCache);
@ -6028,7 +6019,7 @@ public String toString() {
* @throws IOException
*/
Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
String[] cookieTab) throws IOException {
String[] cookieTab) throws IOException {
checkSuperuserPrivilege();
checkOperation(OperationCategory.READ);
readLock();
@ -7089,3 +7080,4 @@ public void logAuditMessage(String message) {
}
}

View File

@ -26,10 +26,7 @@
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.Block;
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.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.MutableBlockCollection;
import org.apache.hadoop.hdfs.server.blockmanagement.*;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.Quota.Counts;
import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeFileUnderConstructionWithSnapshot;
@ -215,7 +212,7 @@ boolean removeLastBlock(Block oldblock) {
*/
@Override
public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeDescriptor[] targets) throws IOException {
DatanodeStorageInfo[] targets) throws IOException {
if (numBlocks() == 0) {
throw new IOException("Failed to set last block: File is empty.");
}

View File

@ -43,8 +43,8 @@
import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.InvalidPathException;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.Path;
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;
@ -61,21 +61,21 @@
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
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.HdfsConstants;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
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.protocol.DSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
@ -91,9 +91,9 @@
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
@ -194,9 +194,9 @@ public NameNodeRpcServer(Configuration conf, NameNode nn)
NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator =
new NamenodeProtocolServerSideTranslatorPB(this);
BlockingService NNPbService = NamenodeProtocolService
BlockingService NNPbService = NamenodeProtocolService
.newReflectiveBlockingService(namenodeProtocolXlator);
RefreshAuthorizationPolicyProtocolServerSideTranslatorPB refreshAuthPolicyXlator =
new RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(this);
BlockingService refreshAuthService = RefreshAuthorizationPolicyProtocolService
@ -216,7 +216,7 @@ public NameNodeRpcServer(Configuration conf, NameNode nn)
new HAServiceProtocolServerSideTranslatorPB(this);
BlockingService haPbService = HAServiceProtocolService
.newReflectiveBlockingService(haServiceProtocolXlator);
WritableRpcEngine.ensureInitialized();
InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
@ -562,7 +562,8 @@ public LocatedBlock addBlock(String src, String clientName,
@Override // ClientProtocol
public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock blk,
final DatanodeInfo[] existings, final DatanodeInfo[] excludes,
final DatanodeInfo[] existings, final String[] existingStorageIDs,
final DatanodeInfo[] excludes,
final int numAdditionalNodes, final String clientName
) throws IOException {
if (LOG.isDebugEnabled()) {
@ -583,8 +584,8 @@ public LocatedBlock getAdditionalDatanode(final String src, final ExtendedBlock
excludeSet.add(node);
}
}
return namesystem.getAdditionalDatanode(src, blk,
existings, excludeSet, numAdditionalNodes, clientName);
return namesystem.getAdditionalDatanode(src, blk, existings,
existingStorageIDs, excludeSet, numAdditionalNodes, clientName);
}
/**
* The client needs to give up on the block.
@ -632,9 +633,9 @@ public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientNam
@Override // ClientProtocol
public void updatePipeline(String clientName, ExtendedBlock oldBlock,
ExtendedBlock newBlock, DatanodeID[] newNodes)
ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
throws IOException {
namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes);
namesystem.updatePipeline(clientName, oldBlock, newBlock, newNodes, newStorageIDs);
}
@Override // DatanodeProtocol
@ -955,24 +956,24 @@ public HeartbeatResponse sendHeartbeat(DatanodeRegistration nodeReg,
StorageReport[] report, int xmitsInProgress, int xceiverCount,
int failedVolumes) throws IOException {
verifyRequest(nodeReg);
return namesystem.handleHeartbeat(nodeReg, report[0].getCapacity(),
report[0].getDfsUsed(), report[0].getRemaining(),
report[0].getBlockPoolUsed(), xceiverCount, xmitsInProgress,
failedVolumes);
return namesystem.handleHeartbeat(nodeReg, report,
xceiverCount, xmitsInProgress, failedVolumes);
}
@Override // DatanodeProtocol
public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
String poolId, StorageBlockReport[] reports) throws IOException {
verifyRequest(nodeReg);
BlockListAsLongs blist = new BlockListAsLongs(reports[0].getBlocks());
if(blockStateChangeLog.isDebugEnabled()) {
blockStateChangeLog.debug("*BLOCK* NameNode.blockReport: "
+ "from " + nodeReg + " " + blist.getNumberOfBlocks()
+ " blocks");
+ "from " + nodeReg + ", reports.length=" + reports.length);
}
final BlockManager bm = namesystem.getBlockManager();
for(StorageBlockReport r : reports) {
final BlockListAsLongs blocks = new BlockListAsLongs(r.getBlocks());
bm.processReport(nodeReg, r.getStorage(), poolId, blocks);
}
namesystem.getBlockManager().processReport(nodeReg, poolId, blist);
if (nn.getFSImage().isUpgradeFinalized() && !nn.isStandbyState())
return new FinalizeCommand(poolId);
return null;
@ -987,8 +988,9 @@ public void blockReceivedAndDeleted(DatanodeRegistration nodeReg, String poolId,
+"from "+nodeReg+" "+receivedAndDeletedBlocks.length
+" blocks.");
}
namesystem.processIncrementalBlockReport(
nodeReg, poolId, receivedAndDeletedBlocks[0].getBlocks());
for(StorageReceivedDeletedBlocks r : receivedAndDeletedBlocks) {
namesystem.processIncrementalBlockReport(nodeReg, poolId, r);
}
}
@Override // DatanodeProtocol
@ -1212,3 +1214,4 @@ public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
return report;
}
}

View File

@ -31,6 +31,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -50,6 +51,7 @@
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
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;
@ -1111,13 +1113,12 @@ public void toXML(XMLOutputter doc) throws IOException {
}
doc.startTag("replicas");
for (final Iterator<DatanodeDescriptor> it = blockManager != null ?
blockManager.datanodeIterator(block) :
Collections.<DatanodeDescriptor>emptyList().iterator();
it.hasNext();) {
for(DatanodeStorageInfo storage : (blockManager != null ?
blockManager.getStorages(block) :
Collections.<DatanodeStorageInfo>emptyList())) {
doc.startTag("replica");
DatanodeDescriptor dd = it.next();
DatanodeDescriptor dd = storage.getDatanodeDescriptor();
doc.startTag("host_name");
doc.pcdata(dd.getHostName());

View File

@ -53,6 +53,7 @@
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@ -61,6 +62,7 @@
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.JspHelper;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@ -160,7 +162,7 @@ private void init(final UserGroupInformation ugi,
static DatanodeInfo chooseDatanode(final NameNode namenode,
final String path, final HttpOpParam.Op op, final long openOffset,
final long blocksize, Configuration conf) throws IOException {
final long blocksize, final Configuration conf) throws IOException {
final BlockManager bm = namenode.getNamesystem().getBlockManager();
if (op == PutOpParam.Op.CREATE) {
@ -168,11 +170,13 @@ static DatanodeInfo chooseDatanode(final NameNode namenode,
final DatanodeDescriptor clientNode = bm.getDatanodeManager(
).getDatanodeByHost(getRemoteAddress());
if (clientNode != null) {
final DatanodeDescriptor[] datanodes = bm.getBlockPlacementPolicy()
final DatanodeStorageInfo[] storages = bm.getBlockPlacementPolicy()
.chooseTarget(path, 1, clientNode,
new ArrayList<DatanodeDescriptor>(), false, null, blocksize);
if (datanodes.length > 0) {
return datanodes[0];
new ArrayList<DatanodeStorageInfo>(), false, null, blocksize,
// TODO: get storage type from the file
StorageType.DEFAULT);
if (storages.length > 0) {
return storages[0].getDatanodeDescriptor();
}
}
} else if (op == GetOpParam.Op.OPEN

View File

@ -24,6 +24,7 @@
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.hdfs.server.blockmanagement.DatanodeStorageInfo;
/****************************************************
* A BlockCommand is an instruction to a datanode
@ -46,9 +47,10 @@ public class BlockCommand extends DatanodeCommand {
*/
public static final long NO_ACK = Long.MAX_VALUE;
String poolId;
Block blocks[];
DatanodeInfo targets[][];
final String poolId;
final Block[] blocks;
final DatanodeInfo[][] targets;
final String[][] targetStorageIDs;
/**
* Create BlockCommand for transferring blocks to another datanode
@ -60,21 +62,26 @@ public BlockCommand(int action, String poolId,
this.poolId = poolId;
blocks = new Block[blocktargetlist.size()];
targets = new DatanodeInfo[blocks.length][];
targetStorageIDs = new String[blocks.length][];
for(int i = 0; i < blocks.length; i++) {
BlockTargetPair p = blocktargetlist.get(i);
blocks[i] = p.block;
targets[i] = p.targets;
targets[i] = DatanodeStorageInfo.toDatanodeInfos(p.targets);
targetStorageIDs[i] = DatanodeStorageInfo.toStorageIDs(p.targets);
}
}
private static final DatanodeInfo[][] EMPTY_TARGET = {};
private static final DatanodeInfo[][] EMPTY_TARGET_DATANODES = {};
private static final String[][] EMPTY_TARGET_STORAGEIDS = {};
/**
* Create BlockCommand for the given action
* @param blocks blocks related to the action
*/
public BlockCommand(int action, String poolId, Block blocks[]) {
this(action, poolId, blocks, EMPTY_TARGET);
this(action, poolId, blocks, EMPTY_TARGET_DATANODES,
EMPTY_TARGET_STORAGEIDS);
}
/**
@ -82,11 +89,12 @@ public BlockCommand(int action, String poolId, Block blocks[]) {
* @param blocks blocks related to the action
*/
public BlockCommand(int action, String poolId, Block[] blocks,
DatanodeInfo[][] targets) {
DatanodeInfo[][] targets, String[][] targetStorageIDs) {
super(action);
this.poolId = poolId;
this.blocks = blocks;
this.targets = targets;
this.targetStorageIDs = targetStorageIDs;
}
public String getBlockPoolId() {
@ -100,4 +108,8 @@ public Block[] getBlocks() {
public DatanodeInfo[][] getTargets() {
return targets;
}
public String[][] getTargetStorageIDs() {
return targetStorageIDs;
}
}

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.hdfs.server.protocol;
import java.util.Arrays;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.Block;
@ -34,12 +36,14 @@ public class BlocksWithLocations {
@InterfaceAudience.Private
@InterfaceStability.Evolving
public static class BlockWithLocations {
Block block;
String storageIDs[];
final Block block;
final String[] datanodeUuids;
final String[] storageIDs;
/** constructor */
public BlockWithLocations(Block block, String[] storageIDs) {
public BlockWithLocations(Block block, String[] datanodeUuids, String[] storageIDs) {
this.block = block;
this.datanodeUuids = datanodeUuids;
this.storageIDs = storageIDs;
}
@ -48,10 +52,30 @@ public Block getBlock() {
return block;
}
/** get the block's locations */
/** get the block's datanode locations */
public String[] getDatanodeUuids() {
return datanodeUuids;
}
/** get the block's storage locations */
public String[] getStorageIDs() {
return storageIDs;
}
@Override
public String toString() {
final StringBuilder b = new StringBuilder();
b.append(block);
if (datanodeUuids.length == 0) {
return b.append("[]").toString();
}
b.append(storageIDs[0]).append('@').append(datanodeUuids[0]);
for(int i = 1; i < datanodeUuids.length; i++) {
b.append(", ").append(storageIDs[i]).append("@").append(datanodeUuids[i]);
}
return b.append("]").toString();
}
}
private BlockWithLocations[] blocks;

View File

@ -82,7 +82,7 @@ public String getAddress() {
public String toString() {
return getClass().getSimpleName()
+ "(" + getIpAddr()
+ ", storageID=" + getStorageID()
+ ", datanodeUuid=" + getDatanodeUuid()
+ ", infoPort=" + getInfoPort()
+ ", ipcPort=" + getIpcPort()
+ ", storageInfo=" + storageInfo

View File

@ -17,6 +17,10 @@
*/
package org.apache.hadoop.hdfs.server.protocol;
import org.apache.hadoop.hdfs.StorageType;
import java.util.UUID;
/**
* Class captures information of a storage in Datanode.
*/
@ -29,18 +33,21 @@ public enum State {
private final String storageID;
private final State state;
private final StorageType storageType;
/**
* Create a storage with {@link State#NORMAL}.
* Create a storage with {@link State#NORMAL} and {@link StorageType#DEFAULT}.
*
* @param storageID
*/
public DatanodeStorage(String storageID) {
this(storageID, State.NORMAL);
this(storageID, State.NORMAL, StorageType.DEFAULT);
}
public DatanodeStorage(String sid, State s) {
storageID = sid;
state = s;
public DatanodeStorage(String sid, State s, StorageType sm) {
this.storageID = sid;
this.state = s;
this.storageType = sm;
}
public String getStorageID() {
@ -50,4 +57,37 @@ public String getStorageID() {
public State getState() {
return state;
}
public StorageType getStorageType() {
return storageType;
}
/**
* Generate new storage ID. The format of this string can be changed
* in the future without requiring that old storage IDs be updated.
*
* @return unique storage ID
*/
public static String generateUuid() {
return "DS-" + UUID.randomUUID();
}
@Override
public boolean equals(Object other){
if (other == this) {
return true;
}
if ((other == null) ||
!(other instanceof DatanodeStorage)) {
return false;
}
DatanodeStorage otherStorage = (DatanodeStorage) other;
return otherStorage.getStorageID().compareTo(getStorageID()) == 0;
}
@Override
public int hashCode() {
return getStorageID().hashCode();
}
}

View File

@ -27,6 +27,8 @@ public class StorageReport {
private final long dfsUsed;
private final long remaining;
private final long blockPoolUsed;
public static final StorageReport[] EMPTY_ARRAY = {};
public StorageReport(String sid, boolean failed, long capacity, long dfsUsed,
long remaining, long bpUsed) {

View File

@ -126,7 +126,7 @@ class ImageLoaderCurrent implements ImageLoader {
new SimpleDateFormat("yyyy-MM-dd HH:mm");
private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
-24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-40, -41, -42, -43, -44, -45, -46, -47 };
-40, -41, -42, -43, -44, -45, -46, -47, -48, -49 };
private int imageVersion = 0;
private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();

View File

@ -276,10 +276,11 @@ private static Map<String, Object> toJsonMap(final DatanodeInfo datanodeinfo) {
return null;
}
// TODO: Fix storageID
final Map<String, Object> m = new TreeMap<String, Object>();
m.put("ipAddr", datanodeinfo.getIpAddr());
m.put("hostName", datanodeinfo.getHostName());
m.put("storageID", datanodeinfo.getStorageID());
m.put("storageID", datanodeinfo.getDatanodeUuid());
m.put("xferPort", datanodeinfo.getXferPort());
m.put("infoPort", datanodeinfo.getInfoPort());
m.put("infoSecurePort", datanodeinfo.getInfoSecurePort());
@ -307,6 +308,7 @@ static DatanodeInfo toDatanodeInfo(final Map<?, ?> m) {
infoSecurePort = 0l; // same as the default value in hdfs.proto
}
// TODO: Fix storageID
return new DatanodeInfo(
(String)m.get("ipAddr"),
(String)m.get("hostName"),
@ -384,7 +386,8 @@ private static LocatedBlock toLocatedBlock(final Map<?, ?> m) throws IOException
final long startOffset = (Long)m.get("startOffset");
final boolean isCorrupt = (Boolean)m.get("isCorrupt");
final LocatedBlock locatedblock = new LocatedBlock(b, locations, startOffset, isCorrupt);
final LocatedBlock locatedblock = new LocatedBlock(b, locations,
null, null, startOffset, isCorrupt);
locatedblock.setBlockToken(toBlockToken((Map<?, ?>)m.get("blockToken")));
return locatedblock;
}

View File

@ -141,6 +141,7 @@ message GetAdditionalDatanodeRequestProto {
repeated DatanodeInfoProto excludes = 4;
required uint32 numAdditionalNodes = 5;
required string clientName = 6;
repeated string existingStorageUuids = 7;
}
message GetAdditionalDatanodeResponseProto {
@ -437,6 +438,7 @@ message UpdatePipelineRequestProto {
required ExtendedBlockProto oldBlock = 2;
required ExtendedBlockProto newBlock = 3;
repeated DatanodeIDProto newNodes = 4;
repeated string storageIDs = 5;
}
message UpdatePipelineResponseProto { // void response

View File

@ -52,8 +52,9 @@ message DatanodeStorageProto {
READ_ONLY = 1;
}
required string storageID = 1; // Unique identifier for the storage
required string storageUuid = 1;
optional StorageState state = 2 [default = NORMAL];
optional StorageTypeProto storageType = 3 [default = DISK];
}
/**
@ -103,10 +104,12 @@ message BlockCommandProto {
INVALIDATE = 2; // Invalidate blocks
SHUTDOWN = 3; // Shutdown the datanode
}
required Action action = 1;
required string blockPoolId = 2;
repeated BlockProto blocks = 3;
repeated DatanodeInfosProto targets = 4;
repeated StorageUuidsProto targetStorageUuids = 5;
}
/**
@ -172,7 +175,7 @@ message HeartbeatRequestProto {
}
message StorageReportProto {
required string storageID = 1;
required string storageUuid = 1;
optional bool failed = 2 [ default = false ];
optional uint64 capacity = 3 [ default = 0 ];
optional uint64 dfsUsed = 4 [ default = 0 ];
@ -250,7 +253,7 @@ message ReceivedDeletedBlockInfoProto {
* List of blocks received and deleted for a storage.
*/
message StorageReceivedDeletedBlocksProto {
required string storageID = 1;
required string storageUuid = 1;
repeated ReceivedDeletedBlockInfoProto blocks = 2;
}

View File

@ -65,7 +65,7 @@ message UpdateReplicaUnderRecoveryRequestProto {
* Response returns updated block information
*/
message UpdateReplicaUnderRecoveryResponseProto {
required string storageID = 1; // ID of the storage that stores replica
optional string storageUuid = 1; // ID of the storage that stores replica
}
/**

View File

@ -50,7 +50,10 @@ message ExtendedBlockProto {
message DatanodeIDProto {
required string ipAddr = 1; // IP address
required string hostName = 2; // hostname
required string storageID = 3; // unique storage id
required string datanodeUuid = 3; // UUID assigned to the Datanode. For
// upgraded clusters this is the same
// as the original StorageID of the
// Datanode.
required uint32 xferPort = 4; // data streaming port
required uint32 infoPort = 5; // datanode http port
required uint32 ipcPort = 6; // ipc server port
@ -114,6 +117,20 @@ message FsPermissionProto {
required uint32 perm = 1; // Actually a short - only 16bits used
}
/**
* Types of recognized storage media.
*/
enum StorageTypeProto {
DISK = 1;
SSD = 2;
}
/**
* A list of storage IDs.
*/
message StorageUuidsProto {
repeated string storageUuids = 1;
}
/**
* A LocatedBlock gives information about a block and its location.
@ -127,6 +144,8 @@ message LocatedBlockProto {
// their locations are not part of this object
required hadoop.common.TokenProto blockToken = 5;
repeated StorageTypeProto storageTypes = 7;
repeated string storageIDs = 8;
}
message DataEncryptionKeyProto {
@ -333,7 +352,8 @@ message BlockProto {
*/
message BlockWithLocationsProto {
required BlockProto block = 1; // Block
repeated string storageIDs = 2; // Datanodes with replicas of the block
repeated string datanodeUuids = 2; // Datanodes with replicas of the block
repeated string storageUuids = 3; // Storages with replicas of the block
}
/**
@ -436,3 +456,4 @@ message SnapshotInfoProto {
// TODO: do we need access time?
}

View File

@ -27,6 +27,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
@ -39,6 +40,7 @@
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
@ -46,6 +48,7 @@
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
@ -797,7 +800,8 @@ public static void setFederatedConfiguration(MiniDFSCluster cluster,
}
private static DatanodeID getDatanodeID(String ipAddr) {
return new DatanodeID(ipAddr, "localhost", "",
return new DatanodeID(ipAddr, "localhost",
UUID.randomUUID().toString(),
DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT,
DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
@ -809,7 +813,8 @@ public static DatanodeID getLocalDatanodeID() {
}
public static DatanodeID getLocalDatanodeID(int port) {
return new DatanodeID("127.0.0.1", "localhost", "",
return new DatanodeID("127.0.0.1", "localhost",
UUID.randomUUID().toString(),
port, port, port, port);
}
@ -831,8 +836,9 @@ public static DatanodeInfo getLocalDatanodeInfo(int port) {
public static DatanodeInfo getDatanodeInfo(String ipAddr,
String host, int port) {
return new DatanodeInfo(new DatanodeID(ipAddr, host, "",
port, DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
return new DatanodeInfo(new DatanodeID(ipAddr, host,
UUID.randomUUID().toString(), port,
DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT));
}
@ -853,9 +859,43 @@ public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
rackLocation);
}
public static DatanodeStorageInfo createDatanodeStorageInfo(
String storageID, String ip) {
return createDatanodeStorageInfo(storageID, ip, "defaultRack");
}
public static DatanodeStorageInfo[] createDatanodeStorageInfos(String[] racks) {
return createDatanodeStorageInfos(racks.length, racks);
}
public static DatanodeStorageInfo[] createDatanodeStorageInfos(int n, String... racks) {
DatanodeStorageInfo[] storages = new DatanodeStorageInfo[n];
for(int i = storages.length; i > 0; ) {
final String storageID = "s" + i;
final String ip = i + "." + i + "." + i + "." + i;
i--;
final String rack = i < racks.length? racks[i]: "defaultRack";
storages[i] = createDatanodeStorageInfo(storageID, ip, rack);
}
return storages;
}
public static DatanodeStorageInfo createDatanodeStorageInfo(
String storageID, String ip, String rack) {
final DatanodeStorage storage = new DatanodeStorage(storageID);
final DatanodeDescriptor dn = BlockManagerTestUtil.getDatanodeDescriptor(ip, rack, storage);
return BlockManagerTestUtil.newDatanodeStorageInfo(dn, storage);
}
public static DatanodeDescriptor[] toDatanodeDescriptor(
DatanodeStorageInfo[] storages) {
DatanodeDescriptor[] datanodes = new DatanodeDescriptor[storages.length];
for(int i = 0; i < datanodes.length; i++) {
datanodes[i] = storages[i].getDatanodeDescriptor();
}
return datanodes;
}
public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
int port, String rackLocation) {
DatanodeID dnId = new DatanodeID(ipAddr, "host", "", port,
DatanodeID dnId = new DatanodeID(ipAddr, "host",
UUID.randomUUID().toString(), port,
DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT);

View File

@ -56,6 +56,7 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Random;
import org.apache.commons.logging.Log;
@ -88,6 +89,7 @@
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.hdfs.web.HftpFileSystem;
@ -125,6 +127,9 @@ public class MiniDFSCluster {
public static final String DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY
= DFS_NAMENODE_SAFEMODE_EXTENSION_KEY + ".testing";
// Changing this value may break some tests that assume it is 2.
public static final int DIRS_PER_DATANODE = 2;
static { DefaultMetricsSystem.setMiniClusterMode(true); }
/**
@ -329,9 +334,10 @@ protected MiniDFSCluster(Builder builder) throws IOException {
builder.nameNodePort, builder.nameNodeHttpPort);
}
LOG.info("starting cluster with " +
builder.nnTopology.countNameNodes() + " namenodes.");
nameNodes = new NameNodeInfo[builder.nnTopology.countNameNodes()];
final int numNameNodes = builder.nnTopology.countNameNodes();
LOG.info("starting cluster: numNameNodes=" + numNameNodes
+ ", numDataNodes=" + builder.numDataNodes);
nameNodes = new NameNodeInfo[numNameNodes];
initMiniDFSCluster(builder.conf,
builder.numDataNodes,
@ -1149,15 +1155,16 @@ public synchronized void startDataNodes(Configuration conf, int numDataNodes,
// Set up datanode address
setupDatanodeAddress(dnConf, setupHostsFile, checkDataNodeAddrConfig);
if (manageDfsDirs) {
File dir1 = getInstanceStorageDir(i, 0);
File dir2 = getInstanceStorageDir(i, 1);
dir1.mkdirs();
dir2.mkdirs();
if (!dir1.isDirectory() || !dir2.isDirectory()) {
throw new IOException("Mkdirs failed to create directory for DataNode "
+ i + ": " + dir1 + " or " + dir2);
StringBuilder sb = new StringBuilder();
for (int j = 0; j < DIRS_PER_DATANODE; ++j) {
File dir = getInstanceStorageDir(i, j);
dir.mkdirs();
if (!dir.isDirectory()) {
throw new IOException("Mkdirs failed to create directory for DataNode " + dir);
}
sb.append((j > 0 ? "," : "") + fileAsURI(dir));
}
String dirs = fileAsURI(dir1) + "," + fileAsURI(dir2);
String dirs = sb.toString();
dnConf.set(DFS_DATANODE_DATA_DIR_KEY, dirs);
conf.set(DFS_DATANODE_DATA_DIR_KEY, dirs);
}
@ -1927,12 +1934,14 @@ private synchronized boolean shouldWait(DatanodeInfo[] dnInfo,
// Wait for expected number of datanodes to start
if (dnInfo.length != numDataNodes) {
LOG.info("dnInfo.length != numDataNodes");
return true;
}
// if one of the data nodes is not fully started, continue to wait
for (DataNodeProperties dn : dataNodes) {
if (!dn.datanode.isDatanodeFullyStarted()) {
LOG.info("!dn.datanode.isDatanodeFullyStarted()");
return true;
}
}
@ -1941,6 +1950,7 @@ private synchronized boolean shouldWait(DatanodeInfo[] dnInfo,
// using (capacity == 0) as proxy.
for (DatanodeInfo dn : dnInfo) {
if (dn.getCapacity() == 0) {
LOG.info("dn.getCapacity() == 0");
return true;
}
}
@ -1948,6 +1958,7 @@ private synchronized boolean shouldWait(DatanodeInfo[] dnInfo,
// If datanode dataset is not initialized then wait
for (DataNodeProperties dn : dataNodes) {
if (DataNodeTestUtils.getFSDataset(dn.datanode) == null) {
LOG.info("DataNodeTestUtils.getFSDataset(dn.datanode) == null");
return true;
}
}
@ -1967,12 +1978,12 @@ public void formatDataNodeDirs() throws IOException {
* @param dataNodeIndex - data node whose block report is desired - the index is same as for getDataNodes()
* @return the block report for the specified data node
*/
public Iterable<Block> getBlockReport(String bpid, int dataNodeIndex) {
public Map<DatanodeStorage, BlockListAsLongs> getBlockReport(String bpid, int dataNodeIndex) {
if (dataNodeIndex < 0 || dataNodeIndex > dataNodes.size()) {
throw new IndexOutOfBoundsException();
}
final DataNode dn = dataNodes.get(dataNodeIndex).datanode;
return DataNodeTestUtils.getFSDataset(dn).getBlockReport(bpid);
return DataNodeTestUtils.getFSDataset(dn).getBlockReports(bpid);
}
@ -1981,11 +1992,12 @@ public Iterable<Block> getBlockReport(String bpid, int dataNodeIndex) {
* @return block reports from all data nodes
* BlockListAsLongs is indexed in the same order as the list of datanodes returned by getDataNodes()
*/
public Iterable<Block>[] getAllBlockReports(String bpid) {
public List<Map<DatanodeStorage, BlockListAsLongs>> getAllBlockReports(String bpid) {
int numDataNodes = dataNodes.size();
Iterable<Block>[] result = new BlockListAsLongs[numDataNodes];
final List<Map<DatanodeStorage, BlockListAsLongs>> result
= new ArrayList<Map<DatanodeStorage, BlockListAsLongs>>(numDataNodes);
for (int i = 0; i < numDataNodes; ++i) {
result[i] = getBlockReport(bpid, i);
result.add(getBlockReport(bpid, i));
}
return result;
}

View File

@ -23,6 +23,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Scanner;
import java.util.concurrent.atomic.AtomicInteger;
@ -35,8 +36,10 @@
import org.apache.hadoop.fs.*;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.tools.DFSAdmin;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.SequenceFile;
@ -1376,11 +1379,14 @@ static List<File> getBlockFiles(MiniDFSCluster cluster) throws IOException {
List<File> files = new ArrayList<File>();
List<DataNode> datanodes = cluster.getDataNodes();
String poolId = cluster.getNamesystem().getBlockPoolId();
Iterable<Block>[] blocks = cluster.getAllBlockReports(poolId);
for(int i = 0; i < blocks.length; i++) {
List<Map<DatanodeStorage, BlockListAsLongs>> blocks = cluster.getAllBlockReports(poolId);
for(int i = 0; i < blocks.size(); i++) {
DataNode dn = datanodes.get(i);
for(Block b : blocks[i]) {
files.add(DataNodeTestUtils.getFile(dn, poolId, b.getBlockId()));
Map<DatanodeStorage, BlockListAsLongs> map = blocks.get(i);
for(Map.Entry<DatanodeStorage, BlockListAsLongs> e : map.entrySet()) {
for(Block b : e.getValue()) {
files.add(DataNodeTestUtils.getFile(dn, poolId, b.getBlockId()));
}
}
}
return files;

View File

@ -237,7 +237,7 @@ boolean isVersionCompatible(StorageData namenodeSd, StorageData datanodeSd) {
* this iterations version 3-tuple
* </pre>
*/
@Test
@Test (timeout=300000)
public void testVersions() throws Exception {
UpgradeUtilities.initialize();
Configuration conf = UpgradeUtilities.initializeStorageStateConf(1,

View File

@ -20,6 +20,8 @@
import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.junit.Assume.assumeTrue;
import java.io.File;
import java.io.IOException;
@ -79,7 +81,8 @@ public void testDataDirectories() throws IOException {
DataNode dn = null;
try {
dn = DataNode.createDataNode(new String[]{}, conf);
} catch(IOException e) {
fail();
} catch(Exception e) {
// expecting exception here
}
if(dn != null)

View File

@ -174,7 +174,7 @@ public void testChangeStorageID() throws Exception {
// register a datanode
DatanodeID dnId = new DatanodeID(DN_IP_ADDR, DN_HOSTNAME,
"fake-storage-id", DN_XFER_PORT, DN_INFO_PORT, DN_INFO_SECURE_PORT,
"fake-datanode-id", DN_XFER_PORT, DN_INFO_PORT, DN_INFO_SECURE_PORT,
DN_IPC_PORT);
long nnCTime = cluster.getNamesystem().getFSImage().getStorage()
.getCTime();
@ -191,7 +191,7 @@ public void testChangeStorageID() throws Exception {
// register the same datanode again with a different storage ID
dnId = new DatanodeID(DN_IP_ADDR, DN_HOSTNAME,
"changed-fake-storage-id", DN_XFER_PORT, DN_INFO_PORT,
"changed-fake-datanode-id", DN_XFER_PORT, DN_INFO_PORT,
DN_INFO_SECURE_PORT, DN_IPC_PORT);
dnReg = new DatanodeRegistration(dnId,
mockStorageInfo, null, VersionInfo.getVersion());
@ -227,7 +227,7 @@ public void testRegistrationWithDifferentSoftwareVersions() throws Exception {
DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
doReturn(HdfsConstants.LAYOUT_VERSION).when(mockDnReg).getVersion();
doReturn(123).when(mockDnReg).getXferPort();
doReturn("fake-storage-id").when(mockDnReg).getStorageID();
doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid();
doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
// Should succeed when software versions are the same.
@ -274,8 +274,7 @@ public void testRegistrationWithDifferentSoftwareVersionsDuringUpgrade()
DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
doReturn(HdfsConstants.LAYOUT_VERSION).when(mockDnReg).getVersion();
doReturn(123).when(mockDnReg).getXferPort();
doReturn("fake-storage-id").when(mockDnReg).getStorageID();
doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid();
doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
// Should succeed when software versions are the same and CTimes are the

View File

@ -158,7 +158,7 @@ public void testArrayOutOfBoundsException() throws Exception {
ns.writeLock();
try {
cluster.getNamesystem().getBlockManager().findAndMarkBlockAsCorrupt(
blk, new DatanodeInfo(dnR), "TEST");
blk, new DatanodeInfo(dnR), "TEST", "STORAGE_ID");
} finally {
ns.writeUnlock();
}

View File

@ -22,6 +22,8 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log;
@ -32,10 +34,12 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.util.Time;
import org.junit.Test;
@ -151,7 +155,7 @@ public void testInjection() throws IOException {
writeFile(cluster.getFileSystem(), testPath, numDataNodes);
waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, 20);
Iterable<Block>[] blocksList = cluster.getAllBlockReports(bpid);
List<Map<DatanodeStorage, BlockListAsLongs>> blocksList = cluster.getAllBlockReports(bpid);
cluster.shutdown();
cluster = null;
@ -172,9 +176,11 @@ public void testInjection() throws IOException {
.build();
cluster.waitActive();
Set<Block> uniqueBlocks = new HashSet<Block>();
for (int i=0; i<blocksList.length; ++i) {
for (Block b : blocksList[i]) {
uniqueBlocks.add(new Block(b));
for(Map<DatanodeStorage, BlockListAsLongs> map : blocksList) {
for(BlockListAsLongs blockList : map.values()) {
for(Block b : blockList) {
uniqueBlocks.add(new Block(b));
}
}
}
// Insert all the blocks in the first data node

View File

@ -151,7 +151,7 @@ public boolean hasSecureChannel() {
public void testAddAndRetrieve() throws Exception {
PeerCache cache = new PeerCache(3, 100000);
DatanodeID dnId = new DatanodeID("192.168.0.1",
"fakehostname", "fake_storage_id",
"fakehostname", "fake_datanode_id",
100, 101, 102, 103);
FakePeer peer = new FakePeer(dnId, false);
cache.put(dnId, peer);
@ -171,7 +171,7 @@ public void testExpiry() throws Exception {
FakePeer peers[] = new FakePeer[CAPACITY];
for (int i = 0; i < CAPACITY; ++i) {
dnIds[i] = new DatanodeID("192.168.0.1",
"fakehostname_" + i, "fake_storage_id",
"fakehostname_" + i, "fake_datanode_id",
100, 101, 102, 103);
peers[i] = new FakePeer(dnIds[i], false);
}
@ -202,7 +202,7 @@ public void testEviction() throws Exception {
FakePeer peers[] = new FakePeer[CAPACITY + 1];
for (int i = 0; i < dnIds.length; ++i) {
dnIds[i] = new DatanodeID("192.168.0.1",
"fakehostname_" + i, "fake_storage_id_" + i,
"fakehostname_" + i, "fake_datanode_id_" + i,
100, 101, 102, 103);
peers[i] = new FakePeer(dnIds[i], false);
}
@ -233,7 +233,7 @@ public void testMultiplePeersWithSameKey() throws Exception {
final int CAPACITY = 3;
PeerCache cache = new PeerCache(CAPACITY, 100000);
DatanodeID dnId = new DatanodeID("192.168.0.1",
"fakehostname", "fake_storage_id",
"fakehostname", "fake_datanode_id",
100, 101, 102, 103);
HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
for (int i = 0; i < CAPACITY; ++i) {
@ -258,7 +258,7 @@ public void testDomainSocketPeers() throws Exception {
final int CAPACITY = 3;
PeerCache cache = new PeerCache(CAPACITY, 100000);
DatanodeID dnId = new DatanodeID("192.168.0.1",
"fakehostname", "fake_storage_id",
"fakehostname", "fake_datanode_id",
100, 101, 102, 103);
HashMultiset<FakePeer> peers = HashMultiset.create(CAPACITY);
for (int i = 0; i < CAPACITY; ++i) {

View File

@ -453,12 +453,14 @@ public static void createDataNodeVersionFile(File[] parent,
*/
public static void createDataNodeVersionFile(File[] parent,
StorageInfo version, String bpid, String bpidToWrite) throws IOException {
DataStorage storage = new DataStorage(version, "doNotCare");
DataStorage storage = new DataStorage(version);
storage.setDatanodeUuid("FixedDatanodeUuid");
File[] versionFiles = new File[parent.length];
for (int i = 0; i < parent.length; i++) {
File versionFile = new File(parent[i], "VERSION");
StorageDirectory sd = new StorageDirectory(parent[i].getParentFile());
storage.createStorageID(sd);
storage.writeProperties(versionFile, sd);
versionFiles[i] = versionFile;
File bpDir = BlockPoolSliceStorage.getBpRoot(bpid, parent[i]);

View File

@ -17,13 +17,16 @@
*/
package org.apache.hadoop.hdfs.protocolPB;
import static org.hamcrest.CoreMatchers.is;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
@ -35,6 +38,7 @@
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeStorageProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockWithLocationsProto;
@ -57,17 +61,9 @@
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
import org.apache.hadoop.hdfs.server.common.StorageInfo;
import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
import org.apache.hadoop.hdfs.server.protocol.*;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
import org.apache.hadoop.security.token.Token;
@ -149,12 +145,18 @@ public void testConvertDatanodeID() {
void compare(DatanodeID dn, DatanodeID dn2) {
assertEquals(dn.getIpAddr(), dn2.getIpAddr());
assertEquals(dn.getHostName(), dn2.getHostName());
assertEquals(dn.getStorageID(), dn2.getStorageID());
assertEquals(dn.getDatanodeUuid(), dn2.getDatanodeUuid());
assertEquals(dn.getXferPort(), dn2.getXferPort());
assertEquals(dn.getInfoPort(), dn2.getInfoPort());
assertEquals(dn.getIpcPort(), dn2.getIpcPort());
}
void compare(DatanodeStorage dns1, DatanodeStorage dns2) {
assertThat(dns2.getStorageID(), is(dns1.getStorageID()));
assertThat(dns2.getState(), is(dns1.getState()));
assertThat(dns2.getStorageType(), is(dns1.getStorageType()));
}
@Test
public void testConvertBlock() {
Block b = new Block(1, 100, 3);
@ -164,8 +166,10 @@ public void testConvertBlock() {
}
private static BlockWithLocations getBlockWithLocations(int bid) {
return new BlockWithLocations(new Block(bid, 0, 1), new String[] { "dn1",
"dn2", "dn3" });
final String[] datanodeUuids = {"dn1", "dn2", "dn3"};
final String[] storageIDs = {"s1", "s2", "s3"};
return new BlockWithLocations(new Block(bid, 0, 1),
datanodeUuids, storageIDs);
}
private void compare(BlockWithLocations locs1, BlockWithLocations locs2) {
@ -428,6 +432,30 @@ private LocatedBlock createLocatedBlock() {
DFSTestUtil.getLocalDatanodeInfo("127.0.0.1", "h3",
AdminStates.NORMAL)
};
String[] storageIDs = {"s1", "s2", "s3"};
StorageType[] media = {
StorageType.DISK,
StorageType.SSD,
StorageType.DISK
};
LocatedBlock lb = new LocatedBlock(
new ExtendedBlock("bp12", 12345, 10, 53),
dnInfos, storageIDs, media, 5, false);
lb.setBlockToken(new Token<BlockTokenIdentifier>(
"identifier".getBytes(), "password".getBytes(), new Text("kind"),
new Text("service")));
return lb;
}
private LocatedBlock createLocatedBlockNoStorageMedia() {
DatanodeInfo[] dnInfos = {
DFSTestUtil.getLocalDatanodeInfo("127.0.0.1", "h1",
AdminStates.DECOMMISSION_INPROGRESS),
DFSTestUtil.getLocalDatanodeInfo("127.0.0.1", "h2",
AdminStates.DECOMMISSIONED),
DFSTestUtil.getLocalDatanodeInfo("127.0.0.1", "h3",
AdminStates.NORMAL)
};
LocatedBlock lb = new LocatedBlock(
new ExtendedBlock("bp12", 12345, 10, 53), dnInfos, 5, false);
lb.setBlockToken(new Token<BlockTokenIdentifier>(
@ -444,6 +472,14 @@ public void testConvertLocatedBlock() {
compare(lb,lb2);
}
@Test
public void testConvertLocatedBlockNoStorageMedia() {
LocatedBlock lb = createLocatedBlockNoStorageMedia();
LocatedBlockProto lbProto = PBHelper.convert(lb);
LocatedBlock lb2 = PBHelper.convert(lbProto);
compare(lb,lb2);
}
@Test
public void testConvertLocatedBlockList() {
ArrayList<LocatedBlock> lbl = new ArrayList<LocatedBlock>();
@ -487,6 +523,16 @@ public void testConvertDatanodeRegistration() {
compare(reg, reg2);
assertEquals(reg.getSoftwareVersion(), reg2.getSoftwareVersion());
}
@Test
public void TestConvertDatanodeStorage() {
DatanodeStorage dns1 = new DatanodeStorage(
"id1", DatanodeStorage.State.NORMAL, StorageType.SSD);
DatanodeStorageProto proto = PBHelper.convert(dns1);
DatanodeStorage dns2 = PBHelper.convert(proto);
compare(dns1, dns2);
}
@Test
public void testConvertBlockCommand() {
@ -496,8 +542,9 @@ public void testConvertBlockCommand() {
dnInfos[0][0] = DFSTestUtil.getLocalDatanodeInfo();
dnInfos[1][0] = DFSTestUtil.getLocalDatanodeInfo();
dnInfos[1][1] = DFSTestUtil.getLocalDatanodeInfo();
String[][] storageIDs = {{"s00"}, {"s10", "s11"}};
BlockCommand bc = new BlockCommand(DatanodeProtocol.DNA_TRANSFER, "bp1",
blocks, dnInfos);
blocks, dnInfos, storageIDs);
BlockCommandProto bcProto = PBHelper.convert(bc);
BlockCommand bc2 = PBHelper.convert(bcProto);
assertEquals(bc.getAction(), bc2.getAction());

View File

@ -31,6 +31,7 @@
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;
@ -50,6 +51,7 @@
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Tool;
import org.apache.log4j.Level;
import org.junit.Test;
/**
@ -58,7 +60,10 @@
public class TestBalancer {
private static final Log LOG = LogFactory.getLog(
"org.apache.hadoop.hdfs.TestBalancer");
static {
((Log4JLogger)Balancer.LOG).getLogger().setLevel(Level.ALL);
}
final static long CAPACITY = 500L;
final static String RACK0 = "/rack0";
final static String RACK1 = "/rack1";
@ -292,6 +297,16 @@ static void waitForBalancer(long totalUsedSpace, long totalCapacity,
} while (!balanced);
}
String long2String(long[] array) {
if (array.length == 0) {
return "<empty>";
}
StringBuilder b = new StringBuilder("[").append(array[0]);
for(int i = 1; i < array.length; i++) {
b.append(", ").append(array[i]);
}
return b.append("]").toString();
}
/** This test start a cluster with specified number of nodes,
* and fills it to be 30% full (with a single file replicated identically
* to all datanodes);
@ -308,6 +323,11 @@ static void waitForBalancer(long totalUsedSpace, long totalCapacity,
*/
private void doTest(Configuration conf, long[] capacities, String[] racks,
long newCapacity, String newRack, boolean useTool) throws Exception {
LOG.info("capacities = " + long2String(capacities));
LOG.info("racks = " + Arrays.asList(racks));
LOG.info("newCapacity= " + newCapacity);
LOG.info("newRack = " + newRack);
LOG.info("useTool = " + useTool);
assertEquals(capacities.length, racks.length);
int numOfDatanodes = capacities.length;
cluster = new MiniDFSCluster.Builder(conf)

View File

@ -18,15 +18,18 @@
package org.apache.hadoop.hdfs.server.blockmanagement;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Set;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.util.Daemon;
import org.junit.Assert;
@ -83,9 +86,8 @@ private static int getNumberOfRacks(final BlockManager blockManager,
final Set<String> rackSet = new HashSet<String>(0);
final Collection<DatanodeDescriptor> corruptNodes =
getCorruptReplicas(blockManager).getNodes(b);
for (Iterator<DatanodeDescriptor> it = blockManager.blocksMap.nodeIterator(b);
it.hasNext();) {
DatanodeDescriptor cur = it.next();
for(DatanodeStorageInfo storage : blockManager.blocksMap.getStorages(b)) {
final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
String rackName = cur.getNetworkLocation();
@ -215,4 +217,52 @@ public static void setWritingPrefersLocalNode(
public static void checkHeartbeat(BlockManager bm) {
bm.getDatanodeManager().getHeartbeatManager().heartbeatCheck();
}
public static DatanodeStorageInfo updateStorage(DatanodeDescriptor dn,
DatanodeStorage s) {
return dn.updateStorage(s);
}
public static DatanodeDescriptor getLocalDatanodeDescriptor(
boolean initializeStorage) {
DatanodeDescriptor dn = new DatanodeDescriptor(DFSTestUtil.getLocalDatanodeID());
if (initializeStorage) {
dn.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid()));
}
return dn;
}
public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
String rackLocation, boolean initializeStorage) {
return getDatanodeDescriptor(ipAddr, rackLocation,
initializeStorage? new DatanodeStorage(DatanodeStorage.generateUuid()): null);
}
public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
String rackLocation, DatanodeStorage storage) {
DatanodeDescriptor dn = DFSTestUtil.getDatanodeDescriptor(ipAddr,
DFSConfigKeys.DFS_DATANODE_DEFAULT_PORT, rackLocation);
if (storage != null) {
dn.updateStorage(storage);
}
return dn;
}
public static DatanodeStorageInfo newDatanodeStorageInfo(
DatanodeDescriptor dn, DatanodeStorage s) {
return new DatanodeStorageInfo(dn, s);
}
public static StorageReport[] getStorageReportsForDatanode(
DatanodeDescriptor dnd) {
ArrayList<StorageReport> reports = new ArrayList<StorageReport>();
for (DatanodeStorageInfo storage : dnd.getStorageInfos()) {
StorageReport report = new StorageReport(
storage.getStorageID(), false, storage.getCapacity(),
storage.getDfsUsed(), storage.getRemaining(),
storage.getBlockPoolUsed());
reports.add(report);
}
return reports.toArray(StorageReport.EMPTY_ARRAY);
}
}

View File

@ -48,7 +48,7 @@ public void testBlockListMoveToHead() throws Exception {
final int MAX_BLOCKS = 10;
DatanodeDescriptor dd = DFSTestUtil.getLocalDatanodeDescriptor();
DatanodeStorageInfo dd = DFSTestUtil.createDatanodeStorageInfo("s1", "1.1.1.1");
ArrayList<Block> blockList = new ArrayList<Block>(MAX_BLOCKS);
ArrayList<BlockInfo> blockInfoList = new ArrayList<BlockInfo>();
int headIndex;
@ -62,7 +62,7 @@ public void testBlockListMoveToHead() throws Exception {
// index of the datanode should be 0
assertEquals("Find datanode should be 0", 0, blockInfoList.get(i)
.findDatanode(dd));
.findStorageInfo(dd));
}
// list length should be equal to the number of blocks we inserted
@ -76,31 +76,31 @@ public void testBlockListMoveToHead() throws Exception {
}
assertEquals("There should be MAX_BLOCK blockInfo's", MAX_BLOCKS, len);
headIndex = dd.getHead().findDatanode(dd);
headIndex = dd.getBlockListHeadForTesting().findStorageInfo(dd);
LOG.info("Moving each block to the head of the list...");
for (int i = 0; i < MAX_BLOCKS; i++) {
curIndex = blockInfoList.get(i).findDatanode(dd);
curIndex = blockInfoList.get(i).findStorageInfo(dd);
headIndex = dd.moveBlockToHead(blockInfoList.get(i), curIndex, headIndex);
// the moved element must be at the head of the list
assertEquals("Block should be at the head of the list now.",
blockInfoList.get(i), dd.getHead());
blockInfoList.get(i), dd.getBlockListHeadForTesting());
}
// move head of the list to the head - this should not change the list
LOG.info("Moving head to the head...");
BlockInfo temp = dd.getHead();
BlockInfo temp = dd.getBlockListHeadForTesting();
curIndex = 0;
headIndex = 0;
dd.moveBlockToHead(temp, curIndex, headIndex);
assertEquals(
"Moving head to the head of the list shopuld not change the list",
temp, dd.getHead());
temp, dd.getBlockListHeadForTesting());
// check all elements of the list against the original blockInfoList
LOG.info("Checking elements of the list...");
temp = dd.getHead();
temp = dd.getBlockListHeadForTesting();
assertNotNull("Head should not be null", temp);
int c = MAX_BLOCKS - 1;
while (temp != null) {
@ -110,15 +110,15 @@ public void testBlockListMoveToHead() throws Exception {
}
LOG.info("Moving random blocks to the head of the list...");
headIndex = dd.getHead().findDatanode(dd);
headIndex = dd.getBlockListHeadForTesting().findStorageInfo(dd);
Random rand = new Random();
for (int i = 0; i < MAX_BLOCKS; i++) {
int j = rand.nextInt(MAX_BLOCKS);
curIndex = blockInfoList.get(j).findDatanode(dd);
curIndex = blockInfoList.get(j).findStorageInfo(dd);
headIndex = dd.moveBlockToHead(blockInfoList.get(j), curIndex, headIndex);
// the moved element must be at the head of the list
assertEquals("Block should be at the head of the list now.",
blockInfoList.get(j), dd.getHead());
blockInfoList.get(j), dd.getBlockListHeadForTesting());
}
}
}

View File

@ -31,18 +31,19 @@
public class TestBlockInfoUnderConstruction {
@Test
public void testInitializeBlockRecovery() throws Exception {
DatanodeDescriptor dd1 = DFSTestUtil.getDatanodeDescriptor("10.10.1.1",
"default");
DatanodeDescriptor dd2 = DFSTestUtil.getDatanodeDescriptor("10.10.1.2",
"default");
DatanodeDescriptor dd3 = DFSTestUtil.getDatanodeDescriptor("10.10.1.3",
"default");
DatanodeStorageInfo s1 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.1", "s1");
DatanodeDescriptor dd1 = s1.getDatanodeDescriptor();
DatanodeStorageInfo s2 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.2", "s2");
DatanodeDescriptor dd2 = s2.getDatanodeDescriptor();
DatanodeStorageInfo s3 = DFSTestUtil.createDatanodeStorageInfo("10.10.1.3", "s3");
DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
3,
BlockUCState.UNDER_CONSTRUCTION,
new DatanodeDescriptor[] {dd1, dd2, dd3});
new DatanodeStorageInfo[] {s1, s2, s3});
// Recovery attempt #1.
long currentTime = System.currentTimeMillis();

View File

@ -22,9 +22,14 @@
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.verify;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.Map.Entry;
@ -39,11 +44,11 @@
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.net.NetworkTopology;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import static org.mockito.Mockito.*;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableList;
@ -51,6 +56,7 @@
import com.google.common.collect.Lists;
public class TestBlockManager {
private DatanodeStorageInfo[] storages;
private List<DatanodeDescriptor> nodes;
private List<DatanodeDescriptor> rackA;
private List<DatanodeDescriptor> rackB;
@ -79,26 +85,29 @@ public void setupMockCluster() throws IOException {
fsn = Mockito.mock(FSNamesystem.class);
Mockito.doReturn(true).when(fsn).hasWriteLock();
bm = new BlockManager(fsn, fsn, conf);
nodes = ImmutableList.of(
DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/rackA"),
DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/rackA"),
DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/rackA"),
DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/rackB"),
DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/rackB"),
DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/rackB")
);
final String[] racks = {
"/rackA",
"/rackA",
"/rackA",
"/rackB",
"/rackB",
"/rackB"};
storages = DFSTestUtil.createDatanodeStorageInfos(racks);
nodes = Arrays.asList(DFSTestUtil.toDatanodeDescriptor(storages));
rackA = nodes.subList(0, 3);
rackB = nodes.subList(3, 6);
}
private void addNodes(Iterable<DatanodeDescriptor> nodesToAdd) {
NetworkTopology cluster = bm.getDatanodeManager().getNetworkTopology();
// construct network topology
for (DatanodeDescriptor dn : nodesToAdd) {
cluster.add(dn);
dn.getStorageInfos()[0].setUtilizationForTesting(
2 * HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
2 * HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L);
dn.updateHeartbeat(
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
BlockManagerTestUtil.getStorageReportsForDatanode(dn), 0, 0);
bm.getDatanodeManager().checkIfClusterIsNowMultiRack(dn);
}
}
@ -123,17 +132,18 @@ public void testBasicReplication() throws Exception {
}
private void doBasicTest(int testIndex) {
List<DatanodeDescriptor> origNodes = getNodes(0, 1);
BlockInfo blockInfo = addBlockOnNodes((long)testIndex, origNodes);
List<DatanodeStorageInfo> origStorages = getStorages(0, 1);
List<DatanodeDescriptor> origNodes = getNodes(origStorages);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo);
assertEquals(2, pipeline.length);
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
origStorages.contains(pipeline[0]));
assertTrue("Destination of replication should be on the other rack. " +
"Was: " + pipeline[1],
rackB.contains(pipeline[1]));
rackB.contains(pipeline[1].getDatanodeDescriptor()));
}
@ -154,21 +164,22 @@ public void testTwoOfThreeNodesDecommissioned() throws Exception {
private void doTestTwoOfThreeNodesDecommissioned(int testIndex) throws Exception {
// Block originally on A1, A2, B1
List<DatanodeDescriptor> origNodes = getNodes(0, 1, 3);
List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
List<DatanodeDescriptor> origNodes = getNodes(origStorages);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
// Decommission two of the nodes (A1, A2)
List<DatanodeDescriptor> decomNodes = startDecommission(0, 1);
DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo);
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
origStorages.contains(pipeline[0]));
assertEquals("Should have three targets", 3, pipeline.length);
boolean foundOneOnRackA = false;
for (int i = 1; i < pipeline.length; i++) {
DatanodeDescriptor target = pipeline[i];
DatanodeDescriptor target = pipeline[i].getDatanodeDescriptor();
if (rackA.contains(target)) {
foundOneOnRackA = true;
}
@ -197,22 +208,23 @@ public void testAllNodesHoldingReplicasDecommissioned() throws Exception {
private void doTestAllNodesHoldingReplicasDecommissioned(int testIndex) throws Exception {
// Block originally on A1, A2, B1
List<DatanodeDescriptor> origNodes = getNodes(0, 1, 3);
List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
List<DatanodeDescriptor> origNodes = getNodes(origStorages);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
// Decommission all of the nodes
List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 3);
DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo);
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
origStorages.contains(pipeline[0]));
assertEquals("Should have three targets", 4, pipeline.length);
boolean foundOneOnRackA = false;
boolean foundOneOnRackB = false;
for (int i = 1; i < pipeline.length; i++) {
DatanodeDescriptor target = pipeline[i];
DatanodeDescriptor target = pipeline[i].getDatanodeDescriptor();
if (rackA.contains(target)) {
foundOneOnRackA = true;
} else if (rackB.contains(target)) {
@ -249,21 +261,22 @@ public void testOneOfTwoRacksDecommissioned() throws Exception {
private void doTestOneOfTwoRacksDecommissioned(int testIndex) throws Exception {
// Block originally on A1, A2, B1
List<DatanodeDescriptor> origNodes = getNodes(0, 1, 3);
List<DatanodeStorageInfo> origStorages = getStorages(0, 1, 3);
List<DatanodeDescriptor> origNodes = getNodes(origStorages);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
// Decommission all of the nodes in rack A
List<DatanodeDescriptor> decomNodes = startDecommission(0, 1, 2);
DatanodeDescriptor[] pipeline = scheduleSingleReplication(blockInfo);
DatanodeStorageInfo[] pipeline = scheduleSingleReplication(blockInfo);
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
origStorages.contains(pipeline[0]));
assertEquals("Should have three targets", 3, pipeline.length);
boolean foundOneOnRackB = false;
for (int i = 1; i < pipeline.length; i++) {
DatanodeDescriptor target = pipeline[i];
DatanodeDescriptor target = pipeline[i].getDatanodeDescriptor();
if (rackB.contains(target)) {
foundOneOnRackB = true;
}
@ -282,11 +295,12 @@ private void doTestOneOfTwoRacksDecommissioned(int testIndex) throws Exception {
// the third off-rack replica.
DatanodeDescriptor rackCNode =
DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/rackC");
rackCNode.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid()));
addNodes(ImmutableList.of(rackCNode));
try {
DatanodeDescriptor[] pipeline2 = scheduleSingleReplication(blockInfo);
DatanodeStorageInfo[] pipeline2 = scheduleSingleReplication(blockInfo);
assertEquals(2, pipeline2.length);
assertEquals(rackCNode, pipeline2[1]);
assertEquals(rackCNode, pipeline2[1].getDatanodeDescriptor());
} finally {
removeNode(rackCNode);
}
@ -307,30 +321,30 @@ public void testSufficientlyReplBlocksUsesNewRack() throws Exception {
private void doTestSufficientlyReplBlocksUsesNewRack(int testIndex) {
// Originally on only nodes in rack A.
List<DatanodeDescriptor> origNodes = rackA;
BlockInfo blockInfo = addBlockOnNodes((long)testIndex, origNodes);
DatanodeDescriptor pipeline[] = scheduleSingleReplication(blockInfo);
BlockInfo blockInfo = addBlockOnNodes(testIndex, origNodes);
DatanodeStorageInfo pipeline[] = scheduleSingleReplication(blockInfo);
assertEquals(2, pipeline.length); // single new copy
assertTrue("Source of replication should be one of the nodes the block " +
"was on. Was: " + pipeline[0],
origNodes.contains(pipeline[0]));
origNodes.contains(pipeline[0].getDatanodeDescriptor()));
assertTrue("Destination of replication should be on the other rack. " +
"Was: " + pipeline[1],
rackB.contains(pipeline[1]));
rackB.contains(pipeline[1].getDatanodeDescriptor()));
}
@Test
public void testBlocksAreNotUnderreplicatedInSingleRack() throws Exception {
List<DatanodeDescriptor> nodes = ImmutableList.of(
DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/rackA"),
DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/rackA"),
DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/rackA"),
DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/rackA"),
DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/rackA"),
DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/rackA")
BlockManagerTestUtil.getDatanodeDescriptor("1.1.1.1", "/rackA", true),
BlockManagerTestUtil.getDatanodeDescriptor("2.2.2.2", "/rackA", true),
BlockManagerTestUtil.getDatanodeDescriptor("3.3.3.3", "/rackA", true),
BlockManagerTestUtil.getDatanodeDescriptor("4.4.4.4", "/rackA", true),
BlockManagerTestUtil.getDatanodeDescriptor("5.5.5.5", "/rackA", true),
BlockManagerTestUtil.getDatanodeDescriptor("6.6.6.6", "/rackA", true)
);
addNodes(nodes);
List<DatanodeDescriptor> origNodes = nodes.subList(0, 3);;
List<DatanodeDescriptor> origNodes = nodes.subList(0, 3);
for (int i = 0; i < NUM_TEST_ITERS; i++) {
doTestSingleRackClusterIsSufficientlyReplicated(i, origNodes);
}
@ -340,7 +354,7 @@ private void doTestSingleRackClusterIsSufficientlyReplicated(int testIndex,
List<DatanodeDescriptor> origNodes)
throws Exception {
assertEquals(0, bm.numOfUnderReplicatedBlocks());
addBlockOnNodes((long)testIndex, origNodes);
addBlockOnNodes(testIndex, origNodes);
bm.processMisReplicatedBlocks();
assertEquals(0, bm.numOfUnderReplicatedBlocks());
}
@ -351,9 +365,11 @@ private void doTestSingleRackClusterIsSufficientlyReplicated(int testIndex,
* pipeline.
*/
private void fulfillPipeline(BlockInfo blockInfo,
DatanodeDescriptor[] pipeline) throws IOException {
DatanodeStorageInfo[] pipeline) throws IOException {
for (int i = 1; i < pipeline.length; i++) {
bm.addBlock(pipeline[i], blockInfo, null);
DatanodeStorageInfo storage = pipeline[i];
bm.addBlock(storage.getDatanodeDescriptor(), storage.getStorageID(), blockInfo, null);
blockInfo.addStorage(storage);
}
}
@ -362,7 +378,9 @@ private BlockInfo blockOnNodes(long blkId, List<DatanodeDescriptor> nodes) {
BlockInfo blockInfo = new BlockInfo(block, 3);
for (DatanodeDescriptor dn : nodes) {
blockInfo.addNode(dn);
for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
blockInfo.addStorage(storage);
}
}
return blockInfo;
}
@ -374,6 +392,22 @@ private List<DatanodeDescriptor> getNodes(int ... indexes) {
}
return ret;
}
private List<DatanodeDescriptor> getNodes(List<DatanodeStorageInfo> storages) {
List<DatanodeDescriptor> ret = Lists.newArrayList();
for (DatanodeStorageInfo s : storages) {
ret.add(s.getDatanodeDescriptor());
}
return ret;
}
private List<DatanodeStorageInfo> getStorages(int ... indexes) {
List<DatanodeStorageInfo> ret = Lists.newArrayList();
for (int idx : indexes) {
ret.add(storages[idx]);
}
return ret;
}
private List<DatanodeDescriptor> startDecommission(int ... indexes) {
List<DatanodeDescriptor> nodes = getNodes(indexes);
@ -392,7 +426,7 @@ private BlockInfo addBlockOnNodes(long blockId, List<DatanodeDescriptor> nodes)
return blockInfo;
}
private DatanodeDescriptor[] scheduleSingleReplication(Block block) {
private DatanodeStorageInfo[] scheduleSingleReplication(Block block) {
// list for priority 1
List<Block> list_p1 = new ArrayList<Block>();
list_p1.add(block);
@ -410,27 +444,29 @@ private DatanodeDescriptor[] scheduleSingleReplication(Block block) {
assertTrue("replication is pending after work is computed",
bm.pendingReplications.getNumReplicas(block) > 0);
LinkedListMultimap<DatanodeDescriptor, BlockTargetPair> repls = getAllPendingReplications();
LinkedListMultimap<DatanodeStorageInfo, BlockTargetPair> repls = getAllPendingReplications();
assertEquals(1, repls.size());
Entry<DatanodeDescriptor, BlockTargetPair> repl =
Entry<DatanodeStorageInfo, BlockTargetPair> repl =
repls.entries().iterator().next();
DatanodeDescriptor[] targets = repl.getValue().targets;
DatanodeStorageInfo[] targets = repl.getValue().targets;
DatanodeDescriptor[] pipeline = new DatanodeDescriptor[1 + targets.length];
DatanodeStorageInfo[] pipeline = new DatanodeStorageInfo[1 + targets.length];
pipeline[0] = repl.getKey();
System.arraycopy(targets, 0, pipeline, 1, targets.length);
return pipeline;
}
private LinkedListMultimap<DatanodeDescriptor, BlockTargetPair> getAllPendingReplications() {
LinkedListMultimap<DatanodeDescriptor, BlockTargetPair> repls =
private LinkedListMultimap<DatanodeStorageInfo, BlockTargetPair> getAllPendingReplications() {
LinkedListMultimap<DatanodeStorageInfo, BlockTargetPair> repls =
LinkedListMultimap.create();
for (DatanodeDescriptor dn : nodes) {
List<BlockTargetPair> thisRepls = dn.getReplicationCommand(10);
if (thisRepls != null) {
repls.putAll(dn, thisRepls);
for(DatanodeStorageInfo storage : dn.getStorageInfos()) {
repls.putAll(storage, thisRepls);
}
}
}
return repls;
@ -453,7 +489,7 @@ public void testHighestPriReplSrcChosenDespiteMaxReplLimit() throws Exception {
addBlockOnNodes(blockId,origNodes.subList(0,1));
List<DatanodeDescriptor> cntNodes = new LinkedList<DatanodeDescriptor>();
List<DatanodeDescriptor> liveNodes = new LinkedList<DatanodeDescriptor>();
List<DatanodeStorageInfo> liveNodes = new LinkedList<DatanodeStorageInfo>();
assertNotNull("Chooses source node for a highest-priority replication"
+ " even if all available source nodes have reached their replication"
@ -476,7 +512,7 @@ public void testHighestPriReplSrcChosenDespiteMaxReplLimit() throws Exception {
UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED));
// Increase the replication count to test replication count > hard limit
DatanodeDescriptor targets[] = { origNodes.get(1) };
DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] };
origNodes.get(0).addBlockToBeReplicated(aBlock, targets);
assertNull("Does not choose a source node for a highest-priority"
@ -492,7 +528,11 @@ public void testHighestPriReplSrcChosenDespiteMaxReplLimit() throws Exception {
@Test
public void testSafeModeIBR() throws Exception {
DatanodeDescriptor node = spy(nodes.get(0));
node.setStorageID("dummy-storage");
DatanodeStorageInfo ds = node.getStorageInfos()[0];
// TODO: Needs to be fixed. DatanodeUuid is not storageID.
node.setDatanodeUuidForTesting(ds.getStorageID());
node.isAlive = true;
DatanodeRegistration nodeReg =
@ -505,35 +545,40 @@ public void testSafeModeIBR() throws Exception {
bm.getDatanodeManager().registerDatanode(nodeReg);
bm.getDatanodeManager().addDatanode(node); // swap in spy
assertEquals(node, bm.getDatanodeManager().getDatanode(node));
assertTrue(node.isFirstBlockReport());
assertEquals(0, ds.getBlockReportCount());
// send block report, should be processed
reset(node);
bm.processReport(node, "pool", new BlockListAsLongs(null, null));
verify(node).receivedBlockReport();
assertFalse(node.isFirstBlockReport());
bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool",
new BlockListAsLongs(null, null));
assertEquals(1, ds.getBlockReportCount());
// send block report again, should NOT be processed
reset(node);
bm.processReport(node, "pool", new BlockListAsLongs(null, null));
verify(node, never()).receivedBlockReport();
assertFalse(node.isFirstBlockReport());
bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool",
new BlockListAsLongs(null, null));
assertEquals(1, ds.getBlockReportCount());
// re-register as if node restarted, should update existing node
bm.getDatanodeManager().removeDatanode(node);
reset(node);
bm.getDatanodeManager().registerDatanode(nodeReg);
verify(node).updateRegInfo(nodeReg);
assertTrue(node.isFirstBlockReport()); // ready for report again
assertEquals(0, ds.getBlockReportCount()); // ready for report again
// send block report, should be processed after restart
reset(node);
bm.processReport(node, "pool", new BlockListAsLongs(null, null));
verify(node).receivedBlockReport();
assertFalse(node.isFirstBlockReport());
bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool",
new BlockListAsLongs(null, null));
assertEquals(1, ds.getBlockReportCount());
}
@Test
public void testSafeModeIBRAfterIncremental() throws Exception {
DatanodeDescriptor node = spy(nodes.get(0));
node.setStorageID("dummy-storage");
DatanodeStorageInfo ds = node.getStorageInfos()[0];
// TODO: Needs to be fixed. DatanodeUuid is not storageID.
node.setDatanodeUuidForTesting(ds.getStorageID());
node.isAlive = true;
DatanodeRegistration nodeReg =
@ -546,12 +591,13 @@ public void testSafeModeIBRAfterIncremental() throws Exception {
bm.getDatanodeManager().registerDatanode(nodeReg);
bm.getDatanodeManager().addDatanode(node); // swap in spy
assertEquals(node, bm.getDatanodeManager().getDatanode(node));
assertTrue(node.isFirstBlockReport());
assertEquals(0, ds.getBlockReportCount());
// send block report while pretending to already have blocks
reset(node);
doReturn(1).when(node).numBlocks();
bm.processReport(node, "pool", new BlockListAsLongs(null, null));
verify(node).receivedBlockReport();
assertFalse(node.isFirstBlockReport());
bm.processReport(node, new DatanodeStorage(ds.getStorageID()), "pool",
new BlockListAsLongs(null, null));
assertEquals(1, ds.getBlockReportCount());
}
}

View File

@ -55,21 +55,24 @@ public void testGetInvalidateBlocks() throws Exception {
@Test
public void testBlocksCounter() throws Exception {
DatanodeDescriptor dd = DFSTestUtil.getLocalDatanodeDescriptor();
DatanodeDescriptor dd = BlockManagerTestUtil.getLocalDatanodeDescriptor(true);
assertEquals(0, dd.numBlocks());
BlockInfo blk = new BlockInfo(new Block(1L), 1);
BlockInfo blk1 = new BlockInfo(new Block(2L), 2);
DatanodeStorageInfo[] storages = dd.getStorageInfos();
assertTrue(storages.length > 0);
final String storageID = storages[0].getStorageID();
// add first block
assertTrue(dd.addBlock(blk));
assertTrue(dd.addBlock(storageID, blk));
assertEquals(1, dd.numBlocks());
// remove a non-existent block
assertFalse(dd.removeBlock(blk1));
assertEquals(1, dd.numBlocks());
// add an existent block
assertFalse(dd.addBlock(blk));
assertFalse(dd.addBlock(storageID, blk));
assertEquals(1, dd.numBlocks());
// add second block
assertTrue(dd.addBlock(blk1));
assertTrue(dd.addBlock(storageID, blk1));
assertEquals(2, dd.numBlocks());
// remove first block
assertTrue(dd.removeBlock(blk));

View File

@ -76,7 +76,7 @@ public void testNumVersionsReportedCorrect() throws IOException {
it.next();
}
DatanodeRegistration toRemove = it.next().getValue();
Log.info("Removing node " + toRemove.getStorageID() + " ip " +
Log.info("Removing node " + toRemove.getDatanodeUuid() + " ip " +
toRemove.getXferAddr() + " version : " + toRemove.getSoftwareVersion());
//Remove that random node
@ -90,7 +90,7 @@ public void testNumVersionsReportedCorrect() throws IOException {
String storageID = "someStorageID" + rng.nextInt(5000);
DatanodeRegistration dr = Mockito.mock(DatanodeRegistration.class);
Mockito.when(dr.getStorageID()).thenReturn(storageID);
Mockito.when(dr.getDatanodeUuid()).thenReturn(storageID);
//If this storageID had already been registered before
if(sIdToDnReg.containsKey(storageID)) {
@ -110,7 +110,7 @@ public void testNumVersionsReportedCorrect() throws IOException {
Mockito.when(dr.getSoftwareVersion()).thenReturn(
"version" + rng.nextInt(5));
Log.info("Registering node storageID: " + dr.getStorageID() +
Log.info("Registering node storageID: " + dr.getDatanodeUuid() +
", version: " + dr.getSoftwareVersion() + ", IP address: "
+ dr.getXferAddr());

View File

@ -20,7 +20,6 @@
import static org.junit.Assert.assertEquals;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -38,6 +37,7 @@
import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.junit.Test;
/**
@ -63,6 +63,8 @@ public void testHeartbeat() throws Exception {
final DatanodeRegistration nodeReg =
DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(0), poolId);
final DatanodeDescriptor dd = NameNodeAdapter.getDatanode(namesystem, nodeReg);
final String storageID = DatanodeStorage.generateUuid();
dd.updateStorage(new DatanodeStorage(storageID));
final int REMAINING_BLOCKS = 1;
final int MAX_REPLICATE_LIMIT =
@ -70,7 +72,7 @@ public void testHeartbeat() throws Exception {
final int MAX_INVALIDATE_LIMIT = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT;
final int MAX_INVALIDATE_BLOCKS = 2*MAX_INVALIDATE_LIMIT+REMAINING_BLOCKS;
final int MAX_REPLICATE_BLOCKS = 2*MAX_REPLICATE_LIMIT+REMAINING_BLOCKS;
final DatanodeDescriptor[] ONE_TARGET = new DatanodeDescriptor[1];
final DatanodeStorageInfo[] ONE_TARGET = {dd.getStorageInfo(storageID)};
try {
namesystem.writeLock();
@ -144,12 +146,15 @@ public void testHeartbeatBlockRecovery() throws Exception {
final DatanodeRegistration nodeReg1 =
DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(0), poolId);
final DatanodeDescriptor dd1 = NameNodeAdapter.getDatanode(namesystem, nodeReg1);
dd1.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid()));
final DatanodeRegistration nodeReg2 =
DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(1), poolId);
final DatanodeDescriptor dd2 = NameNodeAdapter.getDatanode(namesystem, nodeReg2);
dd2.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid()));
final DatanodeRegistration nodeReg3 =
DataNodeTestUtils.getDNRegistrationForBP(cluster.getDataNodes().get(2), poolId);
final DatanodeDescriptor dd3 = NameNodeAdapter.getDatanode(namesystem, nodeReg3);
dd3.updateStorage(new DatanodeStorage(DatanodeStorage.generateUuid()));
try {
namesystem.writeLock();
@ -162,10 +167,13 @@ public void testHeartbeatBlockRecovery() throws Exception {
dd1.setLastUpdate(System.currentTimeMillis());
dd2.setLastUpdate(System.currentTimeMillis());
dd3.setLastUpdate(System.currentTimeMillis());
final DatanodeStorageInfo[] storages = {
dd1.getStorageInfos()[0],
dd2.getStorageInfos()[0],
dd3.getStorageInfos()[0]};
BlockInfoUnderConstruction blockInfo = new BlockInfoUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3,
BlockUCState.UNDER_RECOVERY,
new DatanodeDescriptor[] {dd1, dd2, dd3});
BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo);
DatanodeCommand[] cmds =
NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
@ -187,8 +195,7 @@ public void testHeartbeatBlockRecovery() throws Exception {
dd3.setLastUpdate(System.currentTimeMillis());
blockInfo = new BlockInfoUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3,
BlockUCState.UNDER_RECOVERY,
new DatanodeDescriptor[] {dd1, dd2, dd3});
BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo);
cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
assertEquals(1, cmds.length);
@ -209,8 +216,7 @@ public void testHeartbeatBlockRecovery() throws Exception {
dd3.setLastUpdate(System.currentTimeMillis() - 80 * 1000);
blockInfo = new BlockInfoUnderConstruction(
new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), 3,
BlockUCState.UNDER_RECOVERY,
new DatanodeDescriptor[] {dd1, dd2, dd3});
BlockUCState.UNDER_RECOVERY, storages);
dd1.addBlockToBeRecovered(blockInfo);
cmds = NameNodeAdapter.sendHeartBeat(nodeReg1, dd1, namesystem).getCommands();
assertEquals(1, cmds.length);

View File

@ -20,7 +20,6 @@
import static org.junit.Assert.assertTrue;
import java.util.Collection;
import java.util.Iterator;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.conf.Configuration;
@ -98,12 +97,10 @@ public void testNodeCount() throws Exception {
}
// find out a non-excess node
final Iterator<DatanodeDescriptor> iter = bm.blocksMap
.nodeIterator(block.getLocalBlock());
DatanodeDescriptor nonExcessDN = null;
while (iter.hasNext()) {
DatanodeDescriptor dn = iter.next();
Collection<Block> blocks = bm.excessReplicateMap.get(dn.getStorageID());
for(DatanodeStorageInfo storage : bm.blocksMap.getStorages(block.getLocalBlock())) {
final DatanodeDescriptor dn = storage.getDatanodeDescriptor();
Collection<Block> blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid());
if (blocks == null || !blocks.contains(block.getLocalBlock()) ) {
nonExcessDN = dn;
break;

View File

@ -103,7 +103,10 @@ public void testProcesOverReplicateBlock() throws Exception {
String corruptMachineName = corruptDataNode.getXferAddr();
for (DatanodeDescriptor datanode : hm.getDatanodes()) {
if (!corruptMachineName.equals(datanode.getXferAddr())) {
datanode.updateHeartbeat(100L, 100L, 0L, 100L, 0, 0);
datanode.getStorageInfos()[0].setUtilizationForTesting(100L, 100L, 0, 100L);
datanode.updateHeartbeat(
BlockManagerTestUtil.getStorageReportsForDatanode(datanode),
0, 0);
}
}
@ -155,7 +158,7 @@ public void testChooseReplicaToDelete() throws Exception {
DataNode lastDN = cluster.getDataNodes().get(3);
DatanodeRegistration dnReg = DataNodeTestUtils.getDNRegistrationForBP(
lastDN, namesystem.getBlockPoolId());
String lastDNid = dnReg.getStorageID();
String lastDNid = dnReg.getDatanodeUuid();
final Path fileName = new Path("/foo2");
DFSTestUtil.createFile(fs, fileName, SMALL_FILE_LENGTH, (short)4, 0L);
@ -220,3 +223,4 @@ public void testInvalidateOverReplicatedBlock() throws Exception {
}
}
}

View File

@ -43,8 +43,8 @@ public class TestPendingDataNodeMessages {
@Test
public void testQueues() {
DatanodeDescriptor fakeDN = DFSTestUtil.getLocalDatanodeDescriptor();
msgs.enqueueReportedBlock(fakeDN, block1Gs1, ReplicaState.FINALIZED);
msgs.enqueueReportedBlock(fakeDN, block1Gs2, ReplicaState.FINALIZED);
msgs.enqueueReportedBlock(fakeDN, "STORAGE_ID", block1Gs1, ReplicaState.FINALIZED);
msgs.enqueueReportedBlock(fakeDN, "STORAGE_ID", block1Gs2, ReplicaState.FINALIZED);
assertEquals(2, msgs.count());

View File

@ -43,8 +43,6 @@
import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
import org.junit.Test;
import com.google.common.base.Preconditions;
/**
* This class tests the internals of PendingReplicationBlocks.java,
* as well as how PendingReplicationBlocks acts in BlockManager
@ -54,22 +52,7 @@ public class TestPendingReplication {
private static final int DFS_REPLICATION_INTERVAL = 1;
// Number of datanodes in the cluster
private static final int DATANODE_COUNT = 5;
private DatanodeDescriptor genDatanodeId(int seed) {
seed = seed % 256;
String ip = seed + "." + seed + "." + seed + "." + seed;
return DFSTestUtil.getDatanodeDescriptor(ip, null);
}
private DatanodeDescriptor[] genDatanodes(int number) {
Preconditions.checkArgument(number >= 0);
DatanodeDescriptor[] nodes = new DatanodeDescriptor[number];
for (int i = 0; i < number; i++) {
nodes[i] = genDatanodeId(i);
}
return nodes;
}
@Test
public void testPendingReplication() {
PendingReplicationBlocks pendingReplications;
@ -79,9 +62,13 @@ public void testPendingReplication() {
//
// Add 10 blocks to pendingReplications.
//
for (int i = 0; i < 10; i++) {
DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(10);
for (int i = 0; i < storages.length; i++) {
Block block = new Block(i, i, 0);
pendingReplications.increment(block, genDatanodes(i));
DatanodeStorageInfo[] targets = new DatanodeStorageInfo[i];
System.arraycopy(storages, 0, targets, 0, i);
pendingReplications.increment(block,
DatanodeStorageInfo.toDatanodeDescriptors(targets));
}
assertEquals("Size of pendingReplications ",
10, pendingReplications.size());
@ -91,16 +78,18 @@ public void testPendingReplication() {
// remove one item and reinsert it
//
Block blk = new Block(8, 8, 0);
pendingReplications.decrement(blk, genDatanodeId(7)); // removes one replica
pendingReplications.decrement(blk, storages[7].getDatanodeDescriptor()); // removes one replica
assertEquals("pendingReplications.getNumReplicas ",
7, pendingReplications.getNumReplicas(blk));
for (int i = 0; i < 7; i++) {
// removes all replicas
pendingReplications.decrement(blk, genDatanodeId(i));
pendingReplications.decrement(blk, storages[i].getDatanodeDescriptor());
}
assertTrue(pendingReplications.size() == 9);
pendingReplications.increment(blk, genDatanodes(8));
pendingReplications.increment(blk,
DatanodeStorageInfo.toDatanodeDescriptors(
DFSTestUtil.createDatanodeStorageInfos(8)));
assertTrue(pendingReplications.size() == 10);
//
@ -128,7 +117,9 @@ public void testPendingReplication() {
for (int i = 10; i < 15; i++) {
Block block = new Block(i, i, 0);
pendingReplications.increment(block, genDatanodes(i));
pendingReplications.increment(block,
DatanodeStorageInfo.toDatanodeDescriptors(
DFSTestUtil.createDatanodeStorageInfos(i)));
}
assertTrue(pendingReplications.size() == 15);
@ -210,7 +201,7 @@ public void testBlockReceived() throws Exception {
DatanodeRegistration dnR = datanodes.get(i).getDNRegistrationForBP(
poolId);
StorageReceivedDeletedBlocks[] report = {
new StorageReceivedDeletedBlocks(dnR.getStorageID(),
new StorageReceivedDeletedBlocks("Fake-storage-ID-Ignored",
new ReceivedDeletedBlockInfo[] { new ReceivedDeletedBlockInfo(
blocks[0], BlockStatus.RECEIVED_BLOCK, "") }) };
cluster.getNameNodeRpc().blockReceivedAndDeleted(dnR, poolId, report);
@ -227,7 +218,7 @@ public void testBlockReceived() throws Exception {
DatanodeRegistration dnR = datanodes.get(i).getDNRegistrationForBP(
poolId);
StorageReceivedDeletedBlocks[] report =
{ new StorageReceivedDeletedBlocks(dnR.getStorageID(),
{ new StorageReceivedDeletedBlocks("Fake-storage-ID-Ignored",
new ReceivedDeletedBlockInfo[] { new ReceivedDeletedBlockInfo(
blocks[0], BlockStatus.RECEIVED_BLOCK, "") }) };
cluster.getNameNodeRpc().blockReceivedAndDeleted(dnR, poolId, report);
@ -291,9 +282,9 @@ public void testPendingAndInvalidate() throws Exception {
cluster.getNamesystem().writeLock();
try {
bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[0],
"TEST");
"STORAGE_ID", "TEST");
bm.findAndMarkBlockAsCorrupt(block.getBlock(), block.getLocations()[1],
"TEST");
"STORAGE_ID", "TEST");
} finally {
cluster.getNamesystem().writeUnlock();
}

View File

@ -21,6 +21,9 @@
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.File;
import java.util.ArrayList;
@ -32,6 +35,7 @@
import java.util.Random;
import java.util.Set;
import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -40,11 +44,14 @@
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.LogVerificationAppender;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.hdfs.server.namenode.Namesystem;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.net.Node;
import org.apache.hadoop.test.PathUtils;
@ -58,6 +65,10 @@
import org.junit.rules.ExpectedException;
public class TestReplicationPolicy {
{
((Log4JLogger)BlockPlacementPolicy.LOG).getLogger().setLevel(Level.ALL);
}
private Random random = DFSUtil.getRandom();
private static final int BLOCK_SIZE = 1024;
private static final int NUM_OF_DATANODES = 6;
@ -66,6 +77,7 @@ public class TestReplicationPolicy {
private static BlockPlacementPolicy replicator;
private static final String filename = "/dummyfile.txt";
private static DatanodeDescriptor dataNodes[];
private static DatanodeStorageInfo[] storages;
// The interval for marking a datanode as stale,
private static long staleInterval =
DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT;
@ -73,17 +85,28 @@ public class TestReplicationPolicy {
@Rule
public ExpectedException exception = ExpectedException.none();
private static void updateHeartbeatWithUsage(DatanodeDescriptor dn,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
int xceiverCount, int volFailures) {
dn.getStorageInfos()[0].setUtilizationForTesting(
capacity, dfsUsed, remaining, blockPoolUsed);
dn.updateHeartbeat(
BlockManagerTestUtil.getStorageReportsForDatanode(dn),
xceiverCount, volFailures);
}
@BeforeClass
public static void setupCluster() throws Exception {
Configuration conf = new HdfsConfiguration();
dataNodes = new DatanodeDescriptor[] {
DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1"),
DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1"),
DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r2"),
DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r2"),
DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d2/r3"),
DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d2/r3")
};
final String[] racks = {
"/d1/r1",
"/d1/r1",
"/d1/r2",
"/d1/r2",
"/d2/r3",
"/d2/r3"};
storages = DFSTestUtil.createDatanodeStorageInfos(racks);
dataNodes = DFSTestUtil.toDatanodeDescriptor(storages);
FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
@ -108,12 +131,19 @@ public static void setupCluster() throws Exception {
dataNodes[i]);
}
for (int i=0; i < NUM_OF_DATANODES; i++) {
dataNodes[i].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
}
private static boolean isOnSameRack(DatanodeStorageInfo left, DatanodeStorageInfo right) {
return isOnSameRack(left, right.getDatanodeDescriptor());
}
private static boolean isOnSameRack(DatanodeStorageInfo left, DatanodeDescriptor right) {
return cluster.isOnSameRack(left.getDatanodeDescriptor(), right);
}
/**
* In this testcase, client is dataNodes[0]. So the 1st replica should be
* placed on dataNodes[0], the 2nd replica should be placed on
@ -125,73 +155,73 @@ public static void setupCluster() throws Exception {
*/
@Test
public void testChooseTarget1() throws Exception {
dataNodes[0].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[0],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 4, 0); // overloaded
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0);
assertEquals(targets.length, 0);
targets = chooseTarget(1);
assertEquals(targets.length, 1);
assertEquals(targets[0], dataNodes[0]);
assertEquals(storages[0], targets[0]);
targets = chooseTarget(2);
assertEquals(targets.length, 2);
assertEquals(targets[0], dataNodes[0]);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertEquals(storages[0], targets[0]);
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3);
assertEquals(targets.length, 3);
assertEquals(targets[0], dataNodes[0]);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
assertEquals(storages[0], targets[0]);
assertFalse(isOnSameRack(targets[0], targets[1]));
assertTrue(isOnSameRack(targets[1], targets[2]));
targets = chooseTarget(4);
assertEquals(targets.length, 4);
assertEquals(targets[0], dataNodes[0]);
assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
cluster.isOnSameRack(targets[2], targets[3]));
assertFalse(cluster.isOnSameRack(targets[0], targets[2]));
assertEquals(storages[0], targets[0]);
assertTrue(isOnSameRack(targets[1], targets[2]) ||
isOnSameRack(targets[2], targets[3]));
assertFalse(isOnSameRack(targets[0], targets[2]));
dataNodes[0].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[0],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
private static DatanodeDescriptor[] chooseTarget(int numOfReplicas) {
private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas) {
return chooseTarget(numOfReplicas, dataNodes[0]);
}
private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
DatanodeDescriptor writer) {
return chooseTarget(numOfReplicas, writer,
new ArrayList<DatanodeDescriptor>());
new ArrayList<DatanodeStorageInfo>());
}
private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
List<DatanodeDescriptor> chosenNodes) {
private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
List<DatanodeStorageInfo> chosenNodes) {
return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes);
}
private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
DatanodeDescriptor writer, List<DatanodeDescriptor> chosenNodes) {
private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
DatanodeDescriptor writer, List<DatanodeStorageInfo> chosenNodes) {
return chooseTarget(numOfReplicas, writer, chosenNodes, null);
}
private static DatanodeDescriptor[] chooseTarget(int numOfReplicas,
List<DatanodeDescriptor> chosenNodes, Set<Node> excludedNodes) {
private static DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
List<DatanodeStorageInfo> chosenNodes, Set<Node> excludedNodes) {
return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes, excludedNodes);
}
private static DatanodeDescriptor[] chooseTarget(
private static DatanodeStorageInfo[] chooseTarget(
int numOfReplicas,
DatanodeDescriptor writer,
List<DatanodeDescriptor> chosenNodes,
List<DatanodeStorageInfo> chosenNodes,
Set<Node> excludedNodes) {
return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
false, excludedNodes, BLOCK_SIZE);
false, excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
}
/**
@ -205,8 +235,8 @@ private static DatanodeDescriptor[] chooseTarget(
@Test
public void testChooseTarget2() throws Exception {
Set<Node> excludedNodes;
DatanodeDescriptor[] targets;
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
DatanodeStorageInfo[] targets;
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
excludedNodes = new HashSet<Node>();
excludedNodes.add(dataNodes[1]);
@ -218,49 +248,52 @@ public void testChooseTarget2() throws Exception {
excludedNodes.add(dataNodes[1]);
targets = chooseTarget(1, chosenNodes, excludedNodes);
assertEquals(targets.length, 1);
assertEquals(targets[0], dataNodes[0]);
assertEquals(storages[0], targets[0]);
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.add(dataNodes[1]);
targets = chooseTarget(2, chosenNodes, excludedNodes);
assertEquals(targets.length, 2);
assertEquals(targets[0], dataNodes[0]);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertEquals(storages[0], targets[0]);
assertFalse(isOnSameRack(targets[0], targets[1]));
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.add(dataNodes[1]);
targets = chooseTarget(3, chosenNodes, excludedNodes);
assertEquals(targets.length, 3);
assertEquals(targets[0], dataNodes[0]);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
assertEquals(storages[0], targets[0]);
assertFalse(isOnSameRack(targets[0], targets[1]));
assertTrue(isOnSameRack(targets[1], targets[2]));
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.add(dataNodes[1]);
targets = chooseTarget(4, chosenNodes, excludedNodes);
assertEquals(targets.length, 4);
assertEquals(targets[0], dataNodes[0]);
assertEquals(storages[0], targets[0]);
for(int i=1; i<4; i++) {
assertFalse(cluster.isOnSameRack(targets[0], targets[i]));
assertFalse(isOnSameRack(targets[0], targets[i]));
}
assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
cluster.isOnSameRack(targets[2], targets[3]));
assertFalse(cluster.isOnSameRack(targets[1], targets[3]));
assertTrue(isOnSameRack(targets[1], targets[2]) ||
isOnSameRack(targets[2], targets[3]));
assertFalse(isOnSameRack(targets[1], targets[3]));
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.add(dataNodes[1]);
chosenNodes.add(dataNodes[2]);
chosenNodes.add(storages[2]);
targets = replicator.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
excludedNodes, BLOCK_SIZE);
excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
System.out.println("targets=" + Arrays.asList(targets));
assertEquals(2, targets.length);
//make sure that the chosen node is in the target.
int i = 0;
for (; i < targets.length && !dataNodes[2].equals(targets[i]); i++);
for (; i < targets.length && !storages[2].equals(targets[i]); i++);
assertTrue(i < targets.length);
}
@ -275,40 +308,40 @@ public void testChooseTarget2() throws Exception {
@Test
public void testChooseTarget3() throws Exception {
// make data node 0 to be not qualified to choose
dataNodes[0].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[0],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); // no space
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0);
assertEquals(targets.length, 0);
targets = chooseTarget(1);
assertEquals(targets.length, 1);
assertEquals(targets[0], dataNodes[1]);
assertEquals(storages[1], targets[0]);
targets = chooseTarget(2);
assertEquals(targets.length, 2);
assertEquals(targets[0], dataNodes[1]);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertEquals(storages[1], targets[0]);
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3);
assertEquals(targets.length, 3);
assertEquals(targets[0], dataNodes[1]);
assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertEquals(storages[1], targets[0]);
assertTrue(isOnSameRack(targets[1], targets[2]));
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(4);
assertEquals(targets.length, 4);
assertEquals(targets[0], dataNodes[1]);
assertEquals(storages[1], targets[0]);
for(int i=1; i<4; i++) {
assertFalse(cluster.isOnSameRack(targets[0], targets[i]));
assertFalse(isOnSameRack(targets[0], targets[i]));
}
assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
cluster.isOnSameRack(targets[2], targets[3]));
assertFalse(cluster.isOnSameRack(targets[1], targets[3]));
assertTrue(isOnSameRack(targets[1], targets[2]) ||
isOnSameRack(targets[2], targets[3]));
assertFalse(isOnSameRack(targets[1], targets[3]));
dataNodes[0].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[0],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
@ -325,35 +358,35 @@ public void testChooseTarget3() throws Exception {
public void testChoooseTarget4() throws Exception {
// make data node 0 & 1 to be not qualified to choose: not enough disk space
for(int i=0; i<2; i++) {
dataNodes[i].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
}
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0);
assertEquals(targets.length, 0);
targets = chooseTarget(1);
assertEquals(targets.length, 1);
assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
assertFalse(isOnSameRack(targets[0], dataNodes[0]));
targets = chooseTarget(2);
assertEquals(targets.length, 2);
assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertFalse(isOnSameRack(targets[0], dataNodes[0]));
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3);
assertEquals(targets.length, 3);
for(int i=0; i<3; i++) {
assertFalse(cluster.isOnSameRack(targets[i], dataNodes[0]));
assertFalse(isOnSameRack(targets[i], dataNodes[0]));
}
assertTrue(cluster.isOnSameRack(targets[0], targets[1]) ||
cluster.isOnSameRack(targets[1], targets[2]));
assertFalse(cluster.isOnSameRack(targets[0], targets[2]));
assertTrue(isOnSameRack(targets[0], targets[1]) ||
isOnSameRack(targets[1], targets[2]));
assertFalse(isOnSameRack(targets[0], targets[2]));
for(int i=0; i<2; i++) {
dataNodes[i].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
@ -371,7 +404,7 @@ public void testChooseTarget5() throws Exception {
DatanodeDescriptor writerDesc =
DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r4");
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, writerDesc);
assertEquals(targets.length, 0);
@ -380,12 +413,12 @@ public void testChooseTarget5() throws Exception {
targets = chooseTarget(2, writerDesc);
assertEquals(targets.length, 2);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3, writerDesc);
assertEquals(targets.length, 3);
assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertTrue(isOnSameRack(targets[1], targets[2]));
assertFalse(isOnSameRack(targets[0], targets[1]));
}
/**
@ -415,7 +448,7 @@ public void testChooseTargetWithMoreThanAvailableNodesWithStaleness()
public void testChooseTargetWithMoreThanAvailableNodes() throws Exception {
// make data node 0 & 1 to be not qualified to choose: not enough disk space
for(int i=0; i<2; i++) {
dataNodes[i].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
}
@ -426,7 +459,7 @@ public void testChooseTargetWithMoreThanAvailableNodes() throws Exception {
// try to choose NUM_OF_DATANODES which is more than actually available
// nodes.
DatanodeDescriptor[] targets = chooseTarget(NUM_OF_DATANODES);
DatanodeStorageInfo[] targets = chooseTarget(NUM_OF_DATANODES);
assertEquals(targets.length, NUM_OF_DATANODES - 2);
final List<LoggingEvent> log = appender.getLog();
@ -434,30 +467,42 @@ public void testChooseTargetWithMoreThanAvailableNodes() throws Exception {
assertFalse(log.size() == 0);
final LoggingEvent lastLogEntry = log.get(log.size() - 1);
assertEquals(lastLogEntry.getLevel(), Level.WARN);
assertTrue(Level.WARN.isGreaterOrEqual(lastLogEntry.getLevel()));
// Suppose to place replicas on each node but two data nodes are not
// available for placing replica, so here we expect a short of 2
assertTrue(((String)lastLogEntry.getMessage()).contains("in need of 2"));
for(int i=0; i<2; i++) {
dataNodes[i].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
}
private boolean containsWithinRange(DatanodeDescriptor target,
private boolean containsWithinRange(DatanodeStorageInfo target,
DatanodeDescriptor[] nodes, int startIndex, int endIndex) {
assert startIndex >= 0 && startIndex < nodes.length;
assert endIndex >= startIndex && endIndex < nodes.length;
for (int i = startIndex; i <= endIndex; i++) {
if (nodes[i].equals(target)) {
if (nodes[i].equals(target.getDatanodeDescriptor())) {
return true;
}
}
return false;
}
private boolean containsWithinRange(DatanodeDescriptor target,
DatanodeStorageInfo[] nodes, int startIndex, int endIndex) {
assert startIndex >= 0 && startIndex < nodes.length;
assert endIndex >= startIndex && endIndex < nodes.length;
for (int i = startIndex; i <= endIndex; i++) {
if (nodes[i].getDatanodeDescriptor().equals(target)) {
return true;
}
}
return false;
}
@Test
public void testChooseTargetWithStaleNodes() throws Exception {
// Set dataNodes[0] as stale
@ -466,19 +511,19 @@ public void testChooseTargetWithStaleNodes() throws Exception {
.getDatanodeManager().getHeartbeatManager().heartbeatCheck();
assertTrue(namenode.getNamesystem().getBlockManager()
.getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
// We set the datanode[0] as stale, thus should choose datanode[1] since
// datanode[1] is on the same rack with datanode[0] (writer)
targets = chooseTarget(1);
assertEquals(targets.length, 1);
assertEquals(targets[0], dataNodes[1]);
assertEquals(storages[1], targets[0]);
Set<Node> excludedNodes = new HashSet<Node>();
excludedNodes.add(dataNodes[1]);
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
targets = chooseTarget(1, chosenNodes, excludedNodes);
assertEquals(targets.length, 1);
assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
assertFalse(isOnSameRack(targets[0], dataNodes[0]));
// reset
dataNodes[0].setLastUpdate(Time.now());
@ -503,7 +548,7 @@ public void testChooseTargetWithHalfStaleNodes() throws Exception {
namenode.getNamesystem().getBlockManager()
.getDatanodeManager().getHeartbeatManager().heartbeatCheck();
DatanodeDescriptor[] targets = chooseTarget(0);
DatanodeStorageInfo[] targets = chooseTarget(0);
assertEquals(targets.length, 0);
// Since we have 6 datanodes total, stale nodes should
@ -575,11 +620,12 @@ public void testChooseTargetWithMoreThanHalfStaleNodes() throws Exception {
.getDatanode(miniCluster.getDataNodes().get(0).getDatanodeId());
BlockPlacementPolicy replicator = miniCluster.getNameNode()
.getNamesystem().getBlockManager().getBlockPlacementPolicy();
DatanodeDescriptor[] targets = replicator.chooseTarget(filename, 3,
staleNodeInfo, new ArrayList<DatanodeDescriptor>(), false, null, BLOCK_SIZE);
DatanodeStorageInfo[] targets = replicator.chooseTarget(filename, 3,
staleNodeInfo, new ArrayList<DatanodeStorageInfo>(), false, null,
BLOCK_SIZE, StorageType.DEFAULT);
assertEquals(targets.length, 3);
assertFalse(cluster.isOnSameRack(targets[0], staleNodeInfo));
assertFalse(isOnSameRack(targets[0], staleNodeInfo));
// Step 2. Set more than half of the datanodes as stale
for (int i = 0; i < 4; i++) {
@ -600,10 +646,11 @@ public void testChooseTargetWithMoreThanHalfStaleNodes() throws Exception {
assertFalse(miniCluster.getNameNode().getNamesystem().getBlockManager()
.getDatanodeManager().shouldAvoidStaleDataNodesForWrite());
// Call chooseTarget
targets = replicator.chooseTarget(filename, 3,
staleNodeInfo, new ArrayList<DatanodeDescriptor>(), false, null, BLOCK_SIZE);
targets = replicator.chooseTarget(filename, 3, staleNodeInfo,
new ArrayList<DatanodeStorageInfo>(), false, null, BLOCK_SIZE,
StorageType.DEFAULT);
assertEquals(targets.length, 3);
assertTrue(cluster.isOnSameRack(targets[0], staleNodeInfo));
assertTrue(isOnSameRack(targets[0], staleNodeInfo));
// Step 3. Set 2 stale datanodes back to healthy nodes,
// still have 2 stale nodes
@ -625,7 +672,7 @@ public void testChooseTargetWithMoreThanHalfStaleNodes() throws Exception {
// Call chooseTarget
targets = chooseTarget(3, staleNodeInfo);
assertEquals(targets.length, 3);
assertFalse(cluster.isOnSameRack(targets[0], staleNodeInfo));
assertFalse(isOnSameRack(targets[0], staleNodeInfo));
} finally {
miniCluster.shutdown();
}
@ -640,26 +687,26 @@ public void testChooseTargetWithMoreThanHalfStaleNodes() throws Exception {
*/
@Test
public void testRereplicate1() throws Exception {
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
chosenNodes.add(dataNodes[0]);
DatanodeDescriptor[] targets;
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
chosenNodes.add(storages[0]);
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, chosenNodes);
assertEquals(targets.length, 0);
targets = chooseTarget(1, chosenNodes);
assertEquals(targets.length, 1);
assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(isOnSameRack(targets[0], dataNodes[0]));
targets = chooseTarget(2, chosenNodes);
assertEquals(targets.length, 2);
assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertTrue(isOnSameRack(targets[0], dataNodes[0]));
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3, chosenNodes);
assertEquals(targets.length, 3);
assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(cluster.isOnSameRack(targets[0], targets[2]));
assertTrue(isOnSameRack(targets[0], dataNodes[0]));
assertFalse(isOnSameRack(targets[0], targets[2]));
}
/**
@ -671,22 +718,22 @@ public void testRereplicate1() throws Exception {
*/
@Test
public void testRereplicate2() throws Exception {
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
chosenNodes.add(dataNodes[0]);
chosenNodes.add(dataNodes[1]);
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
chosenNodes.add(storages[0]);
chosenNodes.add(storages[1]);
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, chosenNodes);
assertEquals(targets.length, 0);
targets = chooseTarget(1, chosenNodes);
assertEquals(targets.length, 1);
assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(isOnSameRack(targets[0], dataNodes[0]));
targets = chooseTarget(2, chosenNodes);
assertEquals(targets.length, 2);
assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(cluster.isOnSameRack(dataNodes[0], targets[1]));
assertFalse(isOnSameRack(targets[0], dataNodes[0]));
assertFalse(isOnSameRack(targets[1], dataNodes[0]));
}
/**
@ -698,31 +745,31 @@ public void testRereplicate2() throws Exception {
*/
@Test
public void testRereplicate3() throws Exception {
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
chosenNodes.add(dataNodes[0]);
chosenNodes.add(dataNodes[2]);
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
chosenNodes.add(storages[0]);
chosenNodes.add(storages[2]);
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, chosenNodes);
assertEquals(targets.length, 0);
targets = chooseTarget(1, chosenNodes);
assertEquals(targets.length, 1);
assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(cluster.isOnSameRack(dataNodes[2], targets[0]));
assertTrue(isOnSameRack(targets[0], dataNodes[0]));
assertFalse(isOnSameRack(targets[0], dataNodes[2]));
targets = chooseTarget(1, dataNodes[2], chosenNodes);
assertEquals(targets.length, 1);
assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0]));
assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertTrue(isOnSameRack(targets[0], dataNodes[2]));
assertFalse(isOnSameRack(targets[0], dataNodes[0]));
targets = chooseTarget(2, chosenNodes);
assertEquals(targets.length, 2);
assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertTrue(isOnSameRack(targets[0], dataNodes[0]));
targets = chooseTarget(2, dataNodes[2], chosenNodes);
assertEquals(targets.length, 2);
assertTrue(cluster.isOnSameRack(dataNodes[2], targets[0]));
assertTrue(isOnSameRack(targets[0], dataNodes[2]));
}
/**

View File

@ -33,6 +33,7 @@
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
import org.apache.hadoop.net.NetworkTopology;
@ -54,41 +55,57 @@ public class TestReplicationPolicyWithNodeGroup extends TestCase {
private BlockPlacementPolicy replicator;
private static final String filename = "/dummyfile.txt";
private final static DatanodeDescriptor dataNodes[] = new DatanodeDescriptor[] {
DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1/n1"),
DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1/n1"),
DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r1/n2"),
DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r2/n3"),
DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2/n3"),
DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d1/r2/n4"),
DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r3/n5"),
DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/d2/r3/n6")
};
private final static DatanodeDescriptor dataNodesInBoundaryCase[] =
new DatanodeDescriptor[] {
DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/d1/r1/n1"),
DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/d1/r1/n1"),
DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/d1/r1/n1"),
DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/d1/r1/n2"),
DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/d1/r2/n3"),
DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/d1/r2/n3")
};
private final static DatanodeDescriptor dataNodesInMoreTargetsCase[] =
new DatanodeDescriptor[] {
DFSTestUtil.getDatanodeDescriptor("1.1.1.1", "/r1/n1"),
DFSTestUtil.getDatanodeDescriptor("2.2.2.2", "/r1/n1"),
DFSTestUtil.getDatanodeDescriptor("3.3.3.3", "/r1/n2"),
DFSTestUtil.getDatanodeDescriptor("4.4.4.4", "/r1/n2"),
DFSTestUtil.getDatanodeDescriptor("5.5.5.5", "/r1/n3"),
DFSTestUtil.getDatanodeDescriptor("6.6.6.6", "/r1/n3"),
DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/r2/n4"),
DFSTestUtil.getDatanodeDescriptor("8.8.8.8", "/r2/n4"),
DFSTestUtil.getDatanodeDescriptor("9.9.9.9", "/r2/n5"),
DFSTestUtil.getDatanodeDescriptor("10.10.10.10", "/r2/n5"),
DFSTestUtil.getDatanodeDescriptor("11.11.11.11", "/r2/n6"),
DFSTestUtil.getDatanodeDescriptor("12.12.12.12", "/r2/n6"),
private static final DatanodeStorageInfo[] storages;
private static final DatanodeDescriptor[] dataNodes;
static {
final String[] racks = {
"/d1/r1/n1",
"/d1/r1/n1",
"/d1/r1/n2",
"/d1/r2/n3",
"/d1/r2/n3",
"/d1/r2/n4",
"/d2/r3/n5",
"/d2/r3/n6"
};
storages = DFSTestUtil.createDatanodeStorageInfos(racks);
dataNodes = DFSTestUtil.toDatanodeDescriptor(storages);
}
private static final DatanodeStorageInfo[] storagesInBoundaryCase;
private static final DatanodeDescriptor[] dataNodesInBoundaryCase;
static {
final String[] racksInBoundaryCase = {
"/d1/r1/n1",
"/d1/r1/n1",
"/d1/r1/n1",
"/d1/r1/n2",
"/d1/r2/n3",
"/d1/r2/n3"
};
storagesInBoundaryCase = DFSTestUtil.createDatanodeStorageInfos(racksInBoundaryCase);
dataNodesInBoundaryCase = DFSTestUtil.toDatanodeDescriptor(storagesInBoundaryCase);
}
private static final DatanodeStorageInfo[] storagesInMoreTargetsCase;
private final static DatanodeDescriptor[] dataNodesInMoreTargetsCase;
static {
final String[] racksInMoreTargetsCase = {
"/r1/n1",
"/r1/n1",
"/r1/n2",
"/r1/n2",
"/r1/n3",
"/r1/n3",
"/r2/n4",
"/r2/n4",
"/r2/n5",
"/r2/n5",
"/r2/n6",
"/r2/n6"
};
storagesInMoreTargetsCase = DFSTestUtil.createDatanodeStorageInfos(racksInMoreTargetsCase);
dataNodesInMoreTargetsCase = DFSTestUtil.toDatanodeDescriptor(storagesInMoreTargetsCase);
};
private final static DatanodeDescriptor NODE =
@ -126,9 +143,19 @@ public void tearDown() throws Exception {
namenode.stop();
}
private static void updateHeartbeatWithUsage(DatanodeDescriptor dn,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
int xceiverCount, int volFailures) {
dn.getStorageInfos()[0].setUtilizationForTesting(
capacity, dfsUsed, remaining, blockPoolUsed);
dn.updateHeartbeat(
BlockManagerTestUtil.getStorageReportsForDatanode(dn),
xceiverCount, volFailures);
}
private static void setupDataNodeCapacity() {
for(int i=0; i<NUM_OF_DATANODES; i++) {
dataNodes[i].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
@ -139,11 +166,12 @@ private static void setupDataNodeCapacity() {
* Return false if two targets are found on the same NodeGroup.
*/
private static boolean checkTargetsOnDifferentNodeGroup(
DatanodeDescriptor[] targets) {
DatanodeStorageInfo[] targets) {
if(targets.length == 0)
return true;
Set<String> targetSet = new HashSet<String>();
for(DatanodeDescriptor node:targets) {
for(DatanodeStorageInfo storage:targets) {
final DatanodeDescriptor node = storage.getDatanodeDescriptor();
String nodeGroup = NetworkTopology.getLastHalf(node.getNetworkLocation());
if(targetSet.contains(nodeGroup)) {
return false;
@ -153,34 +181,50 @@ private static boolean checkTargetsOnDifferentNodeGroup(
}
return true;
}
private DatanodeDescriptor[] chooseTarget(int numOfReplicas) {
private boolean isOnSameRack(DatanodeStorageInfo left, DatanodeStorageInfo right) {
return isOnSameRack(left.getDatanodeDescriptor(), right);
}
private boolean isOnSameRack(DatanodeDescriptor left, DatanodeStorageInfo right) {
return cluster.isOnSameRack(left, right.getDatanodeDescriptor());
}
private boolean isOnSameNodeGroup(DatanodeStorageInfo left, DatanodeStorageInfo right) {
return isOnSameNodeGroup(left.getDatanodeDescriptor(), right);
}
private boolean isOnSameNodeGroup(DatanodeDescriptor left, DatanodeStorageInfo right) {
return cluster.isOnSameNodeGroup(left, right.getDatanodeDescriptor());
}
private DatanodeStorageInfo[] chooseTarget(int numOfReplicas) {
return chooseTarget(numOfReplicas, dataNodes[0]);
}
private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
private DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
DatanodeDescriptor writer) {
return chooseTarget(numOfReplicas, writer,
new ArrayList<DatanodeDescriptor>());
new ArrayList<DatanodeStorageInfo>());
}
private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
List<DatanodeDescriptor> chosenNodes) {
private DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
List<DatanodeStorageInfo> chosenNodes) {
return chooseTarget(numOfReplicas, dataNodes[0], chosenNodes);
}
private DatanodeDescriptor[] chooseTarget(int numOfReplicas,
DatanodeDescriptor writer, List<DatanodeDescriptor> chosenNodes) {
private DatanodeStorageInfo[] chooseTarget(int numOfReplicas,
DatanodeDescriptor writer, List<DatanodeStorageInfo> chosenNodes) {
return chooseTarget(numOfReplicas, writer, chosenNodes, null);
}
private DatanodeDescriptor[] chooseTarget(
private DatanodeStorageInfo[] chooseTarget(
int numOfReplicas,
DatanodeDescriptor writer,
List<DatanodeDescriptor> chosenNodes,
List<DatanodeStorageInfo> chosenNodes,
Set<Node> excludedNodes) {
return replicator.chooseTarget(filename, numOfReplicas, writer, chosenNodes,
false, excludedNodes, BLOCK_SIZE);
false, excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
}
/**
@ -193,48 +237,52 @@ private DatanodeDescriptor[] chooseTarget(
* @throws Exception
*/
public void testChooseTarget1() throws Exception {
dataNodes[0].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[0],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 4, 0); // overloaded
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0);
assertEquals(targets.length, 0);
targets = chooseTarget(1);
assertEquals(targets.length, 1);
assertEquals(targets[0], dataNodes[0]);
assertEquals(storages[0], targets[0]);
targets = chooseTarget(2);
assertEquals(targets.length, 2);
assertEquals(targets[0], dataNodes[0]);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertEquals(storages[0], targets[0]);
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3);
assertEquals(targets.length, 3);
assertEquals(targets[0], dataNodes[0]);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
assertFalse(cluster.isOnSameNodeGroup(targets[1], targets[2]));
assertEquals(storages[0], targets[0]);
assertFalse(isOnSameRack(targets[0], targets[1]));
assertTrue(isOnSameRack(targets[1], targets[2]));
assertFalse(isOnSameNodeGroup(targets[1], targets[2]));
targets = chooseTarget(4);
assertEquals(targets.length, 4);
assertEquals(targets[0], dataNodes[0]);
assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
cluster.isOnSameRack(targets[2], targets[3]));
assertFalse(cluster.isOnSameRack(targets[0], targets[2]));
assertEquals(storages[0], targets[0]);
assertTrue(isOnSameRack(targets[1], targets[2]) ||
isOnSameRack(targets[2], targets[3]));
assertFalse(isOnSameRack(targets[0], targets[2]));
// Make sure no more than one replicas are on the same nodegroup
verifyNoTwoTargetsOnSameNodeGroup(targets);
dataNodes[0].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[0],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
private void verifyNoTwoTargetsOnSameNodeGroup(DatanodeDescriptor[] targets) {
private void verifyNoTwoTargetsOnSameNodeGroup(DatanodeStorageInfo[] targets) {
Set<String> nodeGroupSet = new HashSet<String>();
for (DatanodeDescriptor target: targets) {
nodeGroupSet.add(target.getNetworkLocation());
for (DatanodeStorageInfo target: targets) {
nodeGroupSet.add(target.getDatanodeDescriptor().getNetworkLocation());
}
assertEquals(nodeGroupSet.size(), targets.length);
}
@ -248,36 +296,37 @@ private void verifyNoTwoTargetsOnSameNodeGroup(DatanodeDescriptor[] targets) {
* @throws Exception
*/
public void testChooseTarget2() throws Exception {
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
BlockPlacementPolicyDefault repl = (BlockPlacementPolicyDefault)replicator;
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
Set<Node> excludedNodes = new HashSet<Node>();
excludedNodes.add(dataNodes[1]);
targets = repl.chooseTarget(filename, 4, dataNodes[0], chosenNodes, false,
excludedNodes, BLOCK_SIZE);
excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
assertEquals(targets.length, 4);
assertEquals(targets[0], dataNodes[0]);
assertEquals(storages[0], targets[0]);
assertTrue(cluster.isNodeGroupAware());
// Make sure no replicas are on the same nodegroup
for (int i=1;i<4;i++) {
assertFalse(cluster.isOnSameNodeGroup(targets[0], targets[i]));
assertFalse(isOnSameNodeGroup(targets[0], targets[i]));
}
assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
cluster.isOnSameRack(targets[2], targets[3]));
assertFalse(cluster.isOnSameRack(targets[1], targets[3]));
assertTrue(isOnSameRack(targets[1], targets[2]) ||
isOnSameRack(targets[2], targets[3]));
assertFalse(isOnSameRack(targets[1], targets[3]));
excludedNodes.clear();
chosenNodes.clear();
excludedNodes.add(dataNodes[1]);
chosenNodes.add(dataNodes[2]);
chosenNodes.add(storages[2]);
targets = repl.chooseTarget(filename, 1, dataNodes[0], chosenNodes, true,
excludedNodes, BLOCK_SIZE);
excludedNodes, BLOCK_SIZE, StorageType.DEFAULT);
System.out.println("targets=" + Arrays.asList(targets));
assertEquals(2, targets.length);
//make sure that the chosen node is in the target.
int i = 0;
for(; i < targets.length && !dataNodes[2].equals(targets[i]); i++);
for(; i < targets.length && !storages[2].equals(targets[i]); i++);
assertTrue(i < targets.length);
}
@ -291,38 +340,38 @@ public void testChooseTarget2() throws Exception {
*/
public void testChooseTarget3() throws Exception {
// make data node 0 to be not qualified to choose
dataNodes[0].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[0],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0); // no space
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0);
assertEquals(targets.length, 0);
targets = chooseTarget(1);
assertEquals(targets.length, 1);
assertEquals(targets[0], dataNodes[1]);
assertEquals(storages[1], targets[0]);
targets = chooseTarget(2);
assertEquals(targets.length, 2);
assertEquals(targets[0], dataNodes[1]);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertEquals(storages[1], targets[0]);
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3);
assertEquals(targets.length, 3);
assertEquals(targets[0], dataNodes[1]);
assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertEquals(storages[1], targets[0]);
assertTrue(isOnSameRack(targets[1], targets[2]));
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(4);
assertEquals(targets.length, 4);
assertEquals(targets[0], dataNodes[1]);
assertEquals(storages[1], targets[0]);
assertTrue(cluster.isNodeGroupAware());
verifyNoTwoTargetsOnSameNodeGroup(targets);
assertTrue(cluster.isOnSameRack(targets[1], targets[2]) ||
cluster.isOnSameRack(targets[2], targets[3]));
assertTrue(isOnSameRack(targets[1], targets[2]) ||
isOnSameRack(targets[2], targets[3]));
dataNodes[0].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[0],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
@ -339,33 +388,33 @@ public void testChooseTarget3() throws Exception {
public void testChooseTarget4() throws Exception {
// make data node 0-2 to be not qualified to choose: not enough disk space
for(int i=0; i<3; i++) {
dataNodes[i].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
}
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0);
assertEquals(targets.length, 0);
targets = chooseTarget(1);
assertEquals(targets.length, 1);
assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
assertFalse(isOnSameRack(dataNodes[0], targets[0]));
targets = chooseTarget(2);
assertEquals(targets.length, 2);
assertFalse(cluster.isOnSameRack(targets[0], dataNodes[0]));
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertFalse(isOnSameRack(dataNodes[0], targets[0]));
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3);
assertEquals(targets.length, 3);
for(int i=0; i<3; i++) {
assertFalse(cluster.isOnSameRack(targets[i], dataNodes[0]));
assertFalse(isOnSameRack(dataNodes[0], targets[i]));
}
verifyNoTwoTargetsOnSameNodeGroup(targets);
assertTrue(cluster.isOnSameRack(targets[0], targets[1]) ||
cluster.isOnSameRack(targets[1], targets[2]));
assertFalse(cluster.isOnSameRack(targets[0], targets[2]));
assertTrue(isOnSameRack(targets[0], targets[1]) ||
isOnSameRack(targets[1], targets[2]));
assertFalse(isOnSameRack(targets[0], targets[2]));
}
/**
@ -377,7 +426,7 @@ public void testChooseTarget4() throws Exception {
*/
public void testChooseTarget5() throws Exception {
setupDataNodeCapacity();
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, NODE);
assertEquals(targets.length, 0);
@ -386,12 +435,12 @@ public void testChooseTarget5() throws Exception {
targets = chooseTarget(2, NODE);
assertEquals(targets.length, 2);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3, NODE);
assertEquals(targets.length, 3);
assertTrue(cluster.isOnSameRack(targets[1], targets[2]));
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertTrue(isOnSameRack(targets[1], targets[2]));
assertFalse(isOnSameRack(targets[0], targets[1]));
verifyNoTwoTargetsOnSameNodeGroup(targets);
}
@ -404,27 +453,27 @@ public void testChooseTarget5() throws Exception {
*/
public void testRereplicate1() throws Exception {
setupDataNodeCapacity();
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
chosenNodes.add(dataNodes[0]);
DatanodeDescriptor[] targets;
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
chosenNodes.add(storages[0]);
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, chosenNodes);
assertEquals(targets.length, 0);
targets = chooseTarget(1, chosenNodes);
assertEquals(targets.length, 1);
assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(isOnSameRack(dataNodes[0], targets[0]));
targets = chooseTarget(2, chosenNodes);
assertEquals(targets.length, 2);
assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertTrue(isOnSameRack(dataNodes[0], targets[0]));
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3, chosenNodes);
assertEquals(targets.length, 3);
assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(cluster.isOnSameNodeGroup(dataNodes[0], targets[0]));
assertFalse(cluster.isOnSameRack(targets[0], targets[2]));
assertTrue(isOnSameRack(dataNodes[0], targets[0]));
assertFalse(isOnSameNodeGroup(dataNodes[0], targets[0]));
assertFalse(isOnSameRack(targets[0], targets[2]));
}
/**
@ -436,22 +485,22 @@ public void testRereplicate1() throws Exception {
*/
public void testRereplicate2() throws Exception {
setupDataNodeCapacity();
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
chosenNodes.add(dataNodes[0]);
chosenNodes.add(dataNodes[1]);
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
chosenNodes.add(storages[0]);
chosenNodes.add(storages[1]);
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, chosenNodes);
assertEquals(targets.length, 0);
targets = chooseTarget(1, chosenNodes);
assertEquals(targets.length, 1);
assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(isOnSameRack(dataNodes[0], targets[0]));
targets = chooseTarget(2, chosenNodes);
assertEquals(targets.length, 2);
assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]) &&
cluster.isOnSameRack(dataNodes[0], targets[1]));
assertFalse(isOnSameRack(dataNodes[0], targets[0]) &&
isOnSameRack(dataNodes[0], targets[1]));
}
/**
@ -463,33 +512,33 @@ public void testRereplicate2() throws Exception {
*/
public void testRereplicate3() throws Exception {
setupDataNodeCapacity();
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
chosenNodes.add(dataNodes[0]);
chosenNodes.add(dataNodes[3]);
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
chosenNodes.add(storages[0]);
chosenNodes.add(storages[3]);
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, chosenNodes);
assertEquals(targets.length, 0);
targets = chooseTarget(1, chosenNodes);
assertEquals(targets.length, 1);
assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(cluster.isOnSameRack(dataNodes[3], targets[0]));
assertTrue(isOnSameRack(dataNodes[0], targets[0]));
assertFalse(isOnSameRack(dataNodes[3], targets[0]));
targets = chooseTarget(1, dataNodes[3], chosenNodes);
assertEquals(targets.length, 1);
assertTrue(cluster.isOnSameRack(dataNodes[3], targets[0]));
assertFalse(cluster.isOnSameNodeGroup(dataNodes[3], targets[0]));
assertFalse(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertTrue(isOnSameRack(dataNodes[3], targets[0]));
assertFalse(isOnSameNodeGroup(dataNodes[3], targets[0]));
assertFalse(isOnSameRack(dataNodes[0], targets[0]));
targets = chooseTarget(2, chosenNodes);
assertEquals(targets.length, 2);
assertTrue(cluster.isOnSameRack(dataNodes[0], targets[0]));
assertFalse(cluster.isOnSameNodeGroup(dataNodes[0], targets[0]));
assertTrue(isOnSameRack(dataNodes[0], targets[0]));
assertFalse(isOnSameNodeGroup(dataNodes[0], targets[0]));
targets = chooseTarget(2, dataNodes[3], chosenNodes);
assertEquals(targets.length, 2);
assertTrue(cluster.isOnSameRack(dataNodes[3], targets[0]));
assertTrue(isOnSameRack(dataNodes[3], targets[0]));
}
/**
@ -563,16 +612,17 @@ public void testChooseTargetsOnBoundaryTopology() throws Exception {
cluster.add(dataNodesInBoundaryCase[i]);
}
for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
dataNodes[0].updateHeartbeat(
updateHeartbeatWithUsage(dataNodes[0],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE, 0L, 0, 0);
dataNodesInBoundaryCase[i].updateHeartbeat(
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE,
0L, 0, 0);
updateHeartbeatWithUsage(dataNodesInBoundaryCase[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, dataNodesInBoundaryCase[0]);
assertEquals(targets.length, 0);
@ -581,7 +631,7 @@ public void testChooseTargetsOnBoundaryTopology() throws Exception {
targets = chooseTarget(2, dataNodesInBoundaryCase[0]);
assertEquals(targets.length, 2);
assertFalse(cluster.isOnSameRack(targets[0], targets[1]));
assertFalse(isOnSameRack(targets[0], targets[1]));
targets = chooseTarget(3, dataNodesInBoundaryCase[0]);
assertEquals(targets.length, 3);
@ -598,19 +648,17 @@ public void testChooseTargetsOnBoundaryTopology() throws Exception {
@Test
public void testRereplicateOnBoundaryTopology() throws Exception {
for(int i=0; i<NUM_OF_DATANODES_BOUNDARY; i++) {
dataNodesInBoundaryCase[i].updateHeartbeat(
updateHeartbeatWithUsage(dataNodesInBoundaryCase[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
List<DatanodeDescriptor> chosenNodes = new ArrayList<DatanodeDescriptor>();
chosenNodes.add(dataNodesInBoundaryCase[0]);
chosenNodes.add(dataNodesInBoundaryCase[5]);
DatanodeDescriptor[] targets;
List<DatanodeStorageInfo> chosenNodes = new ArrayList<DatanodeStorageInfo>();
chosenNodes.add(storagesInBoundaryCase[0]);
chosenNodes.add(storagesInBoundaryCase[5]);
DatanodeStorageInfo[] targets;
targets = chooseTarget(1, dataNodesInBoundaryCase[0], chosenNodes);
assertFalse(cluster.isOnSameNodeGroup(targets[0],
dataNodesInBoundaryCase[0]));
assertFalse(cluster.isOnSameNodeGroup(targets[0],
dataNodesInBoundaryCase[5]));
assertFalse(isOnSameNodeGroup(dataNodesInBoundaryCase[0], targets[0]));
assertFalse(isOnSameNodeGroup(dataNodesInBoundaryCase[5], targets[0]));
assertTrue(checkTargetsOnDifferentNodeGroup(targets));
}
@ -638,12 +686,12 @@ public void testChooseMoreTargetsThanNodeGroups() throws Exception {
}
for(int i=0; i<NUM_OF_DATANODES_MORE_TARGETS; i++) {
dataNodesInMoreTargetsCase[i].updateHeartbeat(
updateHeartbeatWithUsage(dataNodesInMoreTargetsCase[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0, 0);
}
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
// Test normal case -- 3 replicas
targets = chooseTarget(3, dataNodesInMoreTargetsCase[0]);
assertEquals(targets.length, 3);

View File

@ -47,7 +47,8 @@ public void testSetrepIncWithUnderReplicatedBlocks() throws Exception {
// but the block does not get put into the under-replicated blocks queue
final BlockManager bm = cluster.getNamesystem().getBlockManager();
ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, FILE_PATH);
DatanodeDescriptor dn = bm.blocksMap.nodeIterator(b.getLocalBlock()).next();
DatanodeDescriptor dn = bm.blocksMap.getStorages(b.getLocalBlock())
.iterator().next().getDatanodeDescriptor();
bm.addToInvalidates(b.getLocalBlock(), dn);
Thread.sleep(5000);
bm.blocksMap.removeNode(b.getLocalBlock(), dn);

View File

@ -17,7 +17,6 @@
*/
package org.apache.hadoop.hdfs.server.common;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@ -38,14 +37,19 @@
import javax.xml.parsers.ParserConfigurationException;
import com.google.common.base.Strings;
import static com.google.common.base.Strings.*;
import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
import org.apache.hadoop.hdfs.server.namenode.NameNodeHttpServer;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.hdfs.web.resources.DoAsParam;
import org.apache.hadoop.hdfs.web.resources.UserParam;
import org.apache.hadoop.io.DataInputBuffer;
@ -58,7 +62,6 @@
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSecretManager;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
@ -67,12 +70,6 @@
import org.xml.sax.InputSource;
import org.xml.sax.SAXException;
import static com.google.common.base.Strings.*;
import static org.junit.Assert.*;
import static org.mockito.Mockito.*;
public class TestJspHelper {
private Configuration conf = new HdfsConfiguration();
@ -458,14 +455,28 @@ private void checkUgiFromToken(UserGroupInformation ugi) {
@Test
public void testSortNodeByFields() throws Exception {
DatanodeID dnId1 = new DatanodeID("127.0.0.1", "localhost1", "storage1",
DatanodeID dnId1 = new DatanodeID("127.0.0.1", "localhost1", "datanode1",
1234, 2345, 3456, 4567);
DatanodeID dnId2 = new DatanodeID("127.0.0.2", "localhost2", "storage2",
DatanodeID dnId2 = new DatanodeID("127.0.0.2", "localhost2", "datanode2",
1235, 2346, 3457, 4568);
DatanodeDescriptor dnDesc1 = new DatanodeDescriptor(dnId1, "rack1", 1024,
100, 924, 100, 10, 2);
DatanodeDescriptor dnDesc2 = new DatanodeDescriptor(dnId2, "rack2", 2500,
200, 1848, 200, 20, 1);
// Setup DatanodeDescriptors with one storage each.
DatanodeDescriptor dnDesc1 = new DatanodeDescriptor(dnId1, "rack1");
DatanodeDescriptor dnDesc2 = new DatanodeDescriptor(dnId2, "rack2");
// Update the DatanodeDescriptors with their attached storages.
BlockManagerTestUtil.updateStorage(dnDesc1, new DatanodeStorage("dnStorage1"));
BlockManagerTestUtil.updateStorage(dnDesc2, new DatanodeStorage("dnStorage2"));
StorageReport[] report1 = new StorageReport[] {
new StorageReport("dnStorage1", false, 1024, 100, 924, 100)
};
StorageReport[] report2 = new StorageReport[] {
new StorageReport("dnStorage2", false, 2500, 200, 1848, 200)
};
dnDesc1.updateHeartbeat(report1, 10, 2);
dnDesc2.updateHeartbeat(report2, 20, 1);
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
live.add(dnDesc1);
live.add(dnDesc2);
@ -598,3 +609,4 @@ public void testUpgradeStatusReport() {
MessageFormat.format(EXPECTED_NOTF_PATTERN, version)));
}
}

View File

@ -23,10 +23,10 @@
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import javax.management.NotCompliantMBeanException;
import javax.management.ObjectName;
@ -48,7 +48,9 @@
import org.apache.hadoop.hdfs.server.datanode.fsdataset.RollingLogs;
import org.apache.hadoop.hdfs.server.datanode.metrics.FSDatasetMBean;
import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.util.DataChecksum;
@ -74,7 +76,7 @@ static class Factory extends FsDatasetSpi.Factory<SimulatedFSDataset> {
@Override
public SimulatedFSDataset newInstance(DataNode datanode,
DataStorage storage, Configuration conf) throws IOException {
return new SimulatedFSDataset(datanode, storage, conf);
return new SimulatedFSDataset(storage, conf);
}
@Override
@ -135,6 +137,11 @@ private class BInfo implements ReplicaInPipelineInterface {
}
}
@Override
public String getStorageUuid() {
return storage.getStorageUuid();
}
@Override
synchronized public long getGenerationStamp() {
return theBlock.getGenerationStamp();
@ -318,13 +325,15 @@ void free(long amount) {
private static class SimulatedStorage {
private Map<String, SimulatedBPStorage> map =
new HashMap<String, SimulatedBPStorage>();
private long capacity; // in bytes
private final String storageUuid = "SimulatedStroage-" + DatanodeStorage.generateUuid();
private final long capacity; // in bytes
synchronized long getFree() {
return capacity - getUsed();
}
synchronized long getCapacity() {
long getCapacity() {
return capacity;
}
@ -379,22 +388,33 @@ private SimulatedBPStorage getBPStorage(String bpid) throws IOException {
}
return bpStorage;
}
String getStorageUuid() {
return storageUuid;
}
synchronized StorageReport getStorageReport(String bpid) {
return new StorageReport(getStorageUuid(), false, getCapacity(),
getUsed(), getFree(), map.get(bpid).getUsed());
}
}
private final Map<String, Map<Block, BInfo>> blockMap
= new HashMap<String, Map<Block,BInfo>>();
private final SimulatedStorage storage;
private final String storageId;
private final String datanodeUuid;
public SimulatedFSDataset(DataNode datanode, DataStorage storage,
Configuration conf) {
public SimulatedFSDataset(DataStorage storage, Configuration conf) {
if (storage != null) {
storage.createStorageID(datanode.getXferPort());
this.storageId = storage.getStorageID();
for (int i = 0; i < storage.getNumStorageDirs(); ++i) {
storage.createStorageID(storage.getStorageDir(i));
}
this.datanodeUuid = storage.getDatanodeUuid();
} else {
this.storageId = "unknownStorageId" + new Random().nextInt();
this.datanodeUuid = "SimulatedDatanode-" + DataNode.generateUuid();
}
registerMBean(storageId);
registerMBean(datanodeUuid);
this.storage = new SimulatedStorage(
conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY));
}
@ -451,8 +471,7 @@ public synchronized void unfinalizeBlock(ExtendedBlock b) {
}
}
@Override
public synchronized BlockListAsLongs getBlockReport(String bpid) {
synchronized BlockListAsLongs getBlockReport(String bpid) {
final List<Block> blocks = new ArrayList<Block>();
final Map<Block, BInfo> map = blockMap.get(bpid);
if (map != null) {
@ -465,6 +484,12 @@ public synchronized BlockListAsLongs getBlockReport(String bpid) {
return new BlockListAsLongs(blocks, null);
}
@Override
public synchronized Map<DatanodeStorage, BlockListAsLongs> getBlockReports(
String bpid) {
return Collections.singletonMap(new DatanodeStorage(storage.storageUuid), getBlockReport(bpid));
}
@Override // FSDatasetMBean
public long getCapacity() {
return storage.getCapacity();
@ -619,7 +644,7 @@ public synchronized ReplicaInPipelineInterface recoverAppend(ExtendedBlock b,
}
@Override // FsDatasetSpi
public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
public String recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
throws IOException {
final Map<Block, BInfo> map = getMap(b.getBlockPoolId());
BInfo binfo = map.get(b.getLocalBlock());
@ -633,6 +658,7 @@ public void recoverClose(ExtendedBlock b, long newGS, long expectedBlockLen)
map.remove(b.getLocalBlock());
binfo.theBlock.setGenerationStamp(newGS);
map.put(binfo.theBlock, binfo);
return binfo.getStorageUuid();
}
@Override // FsDatasetSpi
@ -889,7 +915,7 @@ public void shutdown() {
@Override
public String getStorageInfo() {
return "Simulated FSDataset-" + storageId;
return "Simulated FSDataset-" + datanodeUuid;
}
@Override
@ -916,7 +942,8 @@ public ReplicaRecoveryInfo initReplicaRecovery(RecoveringBlock rBlock)
public String updateReplicaUnderRecovery(ExtendedBlock oldBlock,
long recoveryId,
long newlength) {
return storageId;
// Caller does not care about the exact Storage UUID returned.
return datanodeUuid;
}
@Override // FsDatasetSpi
@ -970,11 +997,6 @@ public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
throw new UnsupportedOperationException();
}
@Override
public String[] getBlockPoolList() {
throw new UnsupportedOperationException();
}
@Override
public void checkAndUpdate(String bpid, long blockId, File diskFile,
File diskMetaFile, FsVolumeSpi vol) {
@ -987,7 +1009,12 @@ public List<FsVolumeSpi> getVolumes() {
}
@Override
public List<Block> getFinalizedBlocks(String bpid) {
public StorageReport[] getStorageReports(String bpid) {
return new StorageReport[] {storage.getStorageReport(bpid)};
}
@Override
public List<FinalizedReplica> getFinalizedBlocks(String bpid) {
throw new UnsupportedOperationException();
}
@ -1006,3 +1033,4 @@ public FsVolumeSpi getVolume(ExtendedBlock b) {
throw new UnsupportedOperationException();
}
}

View File

@ -104,7 +104,7 @@ public void setupMocks() throws Exception {
.when(mockDn).getMetrics();
// Set up a simulated dataset with our fake BP
mockFSDataset = Mockito.spy(new SimulatedFSDataset(null, null, conf));
mockFSDataset = Mockito.spy(new SimulatedFSDataset(null, conf));
mockFSDataset.addBlockPool(FAKE_BPID, conf);
// Wire the dataset to the DN.
@ -178,7 +178,7 @@ public void testBasicFunctionality() throws Exception {
waitForBlockReport(mockNN2);
// When we receive a block, it should report it to both NNs
bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, "");
bpos.notifyNamenodeReceivedBlock(FAKE_BLOCK, "", "");
ReceivedDeletedBlockInfo[] ret = waitForBlockReceived(FAKE_BLOCK, mockNN1);
assertEquals(1, ret.length);

View File

@ -35,6 +35,7 @@
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
@ -120,7 +121,7 @@ public class TestBlockRecovery {
* @throws IOException
*/
@Before
public void startUp() throws IOException {
public void startUp() throws IOException, URISyntaxException {
conf = new HdfsConfiguration();
conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, DATA_DIR);
conf.set(DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY, "0.0.0.0:0");
@ -129,11 +130,12 @@ public void startUp() throws IOException {
conf.setInt(CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0);
FileSystem.setDefaultUri(conf,
"hdfs://" + NN_ADDR.getHostName() + ":" + NN_ADDR.getPort());
ArrayList<File> dirs = new ArrayList<File>();
ArrayList<StorageLocation> locations = new ArrayList<StorageLocation>();
File dataDir = new File(DATA_DIR);
FileUtil.fullyDelete(dataDir);
dataDir.mkdirs();
dirs.add(dataDir);
StorageLocation location = StorageLocation.parse(dataDir.getPath());
locations.add(location);
final DatanodeProtocolClientSideTranslatorPB namenode =
mock(DatanodeProtocolClientSideTranslatorPB.class);
@ -159,7 +161,7 @@ public DatanodeRegistration answer(InvocationOnMock invocation)
new DatanodeCommand[0],
new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1)));
dn = new DataNode(conf, dirs, null) {
dn = new DataNode(conf, locations, null) {
@Override
DatanodeProtocolClientSideTranslatorPB connectToNN(
InetSocketAddress nnAddr) throws IOException {

Some files were not shown because too many files have changed in this diff Show More