svn merge --reintegrate https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2832 for merging Heterogeneous Storage feature branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1550363 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Arpit Agarwal 2013-12-12 07:17:51 +00:00
commit d887c6a286
119 changed files with 3949 additions and 2222 deletions

View File

@ -13,6 +13,10 @@ Trunk (Unreleased)
HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
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-4665. Move TestNetworkTopologyWithNodeGroup to common.
@ -440,6 +444,131 @@ Trunk (Unreleased)
HDFS-5626. dfsadmin -report shows incorrect cache values. (cmccabe)
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.4.0 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -1347,7 +1347,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
* 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

@ -312,6 +312,7 @@ public class DFSOutputStream extends FSOutputSummer
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(
@ -402,7 +403,7 @@ public class DFSOutputStream extends FSOutputSummer
}
// 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 " +
@ -411,6 +412,14 @@ public class DFSOutputStream extends FSOutputSummer
}
}
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;
@ -434,7 +443,7 @@ public class DFSOutputStream extends FSOutputSummer
this.setName("DataStreamer for file " + src);
closeResponder();
closeStream();
nodes = null;
setPipeline(null, null);
stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
}
@ -503,7 +512,7 @@ public class DFSOutputStream extends FSOutputSummer
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()) {
@ -917,9 +926,10 @@ public class DFSOutputStream extends FSOutputSummer
//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);
@ -1019,7 +1029,14 @@ public class DFSOutputStream extends FSOutputSummer
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;
@ -1055,7 +1072,8 @@ public class DFSOutputStream extends FSOutputSummer
// 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;
}
@ -1068,7 +1086,7 @@ public class DFSOutputStream extends FSOutputSummer
* 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;
@ -1110,7 +1128,7 @@ public class DFSOutputStream extends FSOutputSummer
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 class DFSUtil {
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 @@ package org.apache.hadoop.hdfs.protocol;
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 @@ public class BlockListAsLongs implements Iterable<Block> {
}
/**
* 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

@ -354,7 +354,8 @@ public interface ClientProtocol {
*/
@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;
@ -983,7 +984,7 @@ public interface ClientProtocol {
*/
@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 @@ package org.apache.hadoop.hdfs.protocol;
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 class DatanodeID implements Comparable<DatanodeID> {
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 class DatanodeID implements Comparable<DatanodeID> {
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 class DatanodeID implements Comparable<DatanodeID> {
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

@ -115,7 +115,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
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,
cacheCapacity, cacheUsed, lastUpdate, xceiverCount, location,
@ -124,13 +124,13 @@ public class DatanodeInfo extends DatanodeID implements Node {
/** 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 cacheCapacity, final long cacheUsed,
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

@ -107,7 +107,10 @@ public class LayoutVersion {
"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"),
CACHING(-48, "Support for cache pools and path-based caching");
CACHING(-48, "Support for cache pools and path-based caching"),
ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
+ " Use distinct StorageUuid per storage directory.");
final int lv;
final int ancestorLV;
@ -248,3 +251,4 @@ public class LayoutVersion {
throw new AssertionError("All layout versions are reserved.");
}
}

View File

@ -21,7 +21,9 @@ import java.util.List;
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;
import com.google.common.base.Preconditions;
@ -40,6 +42,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
@ -54,20 +60,34 @@ public class LocatedBlock {
private static final DatanodeInfo[] EMPTY_LOCS = new DatanodeInfo[0];
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
this(b, locs, -1); // startOffset is unknown
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset) {
this(b, locs, startOffset, false);
this(b, locs, -1, false); // startOffset is unknown
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
boolean corrupt) {
this(b, locs, startOffset, corrupt, EMPTY_LOCS);
this(b, locs, null, null, startOffset, corrupt, EMPTY_LOCS);
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset,
boolean corrupt, DatanodeInfo[] cachedLocs) {
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, EMPTY_LOCS);
}
public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
long startOffset, boolean corrupt) {
this(b, DatanodeStorageInfo.toDatanodeInfos(storages),
DatanodeStorageInfo.toStorageIDs(storages),
DatanodeStorageInfo.toStorageTypes(storages),
startOffset, corrupt, EMPTY_LOCS); // startOffset is unknown
}
public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, String[] storageIDs,
StorageType[] storageTypes, long startOffset,
boolean corrupt, DatanodeInfo[] cachedLocs) {
this.b = b;
this.offset = startOffset;
this.corrupt = corrupt;
@ -76,6 +96,8 @@ public class LocatedBlock {
} else {
this.locs = locs;
}
this.storageIDs = storageIDs;
this.storageTypes = storageTypes;
Preconditions.checkArgument(cachedLocs != null,
"cachedLocs should not be null, use a different constructor");
if (cachedLocs.length == 0) {
@ -100,7 +122,15 @@ public class LocatedBlock {
public DatanodeInfo[] getLocations() {
return locs;
}
public StorageType[] getStorageTypes() {
return storageTypes;
}
public String[] getStorageIDs() {
return storageIDs;
}
public long getStartOffset() {
return offset;
}
@ -161,3 +191,4 @@ public class LocatedBlock {
+ "}";
}
}

View File

@ -51,7 +51,7 @@ public class UnregisteredNodeException extends IOException {
*/
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

@ -425,14 +425,17 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
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();
@ -833,10 +836,12 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
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

@ -351,7 +351,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
@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 {
@ -360,6 +361,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
.setSrc(src)
.setBlk(PBHelper.convert(blk))
.addAllExistings(PBHelper.convert(existings))
.addAllExistingStorageUuids(Arrays.asList(existingStorageIDs))
.addAllExcludes(PBHelper.convert(excludes))
.setNumAdditionalNodes(numAdditionalNodes)
.setClientName(clientName)
@ -796,12 +798,13 @@ public class ClientNamenodeProtocolTranslatorPB implements
@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

@ -245,7 +245,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
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

@ -42,7 +42,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlo
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;
@ -102,14 +101,8 @@ public class DatanodeProtocolServerSideTranslatorPB implements
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.getCacheCapacity(), request.getCacheUsed(),
request.getXmitsInProgress(),
@ -198,7 +191,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
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 class InterDatanodeProtocolServerSideTranslatorPB implements
throw new ServiceException(e);
}
return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
.setStorageID(storageID).build();
.setStorageUuid(storageID).build();
}
}

View File

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

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.ha.proto.HAServiceProtocolProtos;
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.CacheDirectiveEntry;
import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
@ -52,12 +53,12 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
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.CacheDirectiveEntryProto;
@ -122,6 +123,8 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
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;
@ -242,17 +245,20 @@ public class PBHelper {
// 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();
@ -294,12 +300,16 @@ public class PBHelper {
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) {
@ -601,6 +611,17 @@ public class PBHelper {
"Found additional cached replica locations that are not in the set of"
+ " storage-backed locations!");
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();
@ -613,6 +634,25 @@ public class PBHelper {
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]);
}
// Set values from the isCached list, re-using references from loc
List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
List<Boolean> isCachedList = proto.getIsCachedList();
@ -623,7 +663,7 @@ public class PBHelper {
}
LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
proto.getOffset(), proto.getCorrupt(),
storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
cachedLocs.toArray(new DatanodeInfo[0]));
lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
@ -766,7 +806,8 @@ public class PBHelper {
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();
}
@ -799,6 +840,15 @@ public class PBHelper {
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) {
@ -878,6 +928,14 @@ public class PBHelper {
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:
@ -892,7 +950,8 @@ public class PBHelper {
default:
throw new AssertionError("Unknown action type: " + blkCmd.getAction());
}
return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets);
return new BlockCommand(action, blkCmd.getBlockPoolId(), blocks, targets,
targetStorageIDs);
}
public static BlockIdCommand convert(BlockIdCommandProto blkIdCmd) {
@ -1422,11 +1481,12 @@ public class PBHelper {
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;
@ -1436,11 +1496,26 @@ public class PBHelper {
}
}
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;
@ -1450,14 +1525,50 @@ public class PBHelper {
}
}
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) {
StorageReportProto.Builder builder = StorageReportProto.newBuilder()
.setBlockPoolUsed(r.getBlockPoolUsed()).setCapacity(r.getCapacity())
.setDfsUsed(r.getDfsUsed()).setRemaining(r.getRemaining())
.setStorageID(r.getStorageID());
.setStorageUuid(r.getStorageID());
return builder.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) {
int lv = info.hasLayoutVersion() ? info.getLayoutVersion() : 0;
int nsID = info.hasNamespaceID() ? info.getNamespaceID() : 0;
@ -1756,3 +1867,4 @@ public class PBHelper {
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 @@ public class Balancer {
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 @@ public class Balancer {
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 @@ public class Balancer {
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 @@ public class Balancer {
@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 @@ public class Balancer {
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 @@ public class Balancer {
/* 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 @@ public class Balancer {
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 @@ public class Balancer {
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 @@ public class Balancer {
* 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 @@ public class Balancer {
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 @@ public class Balancer {
// 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 @@ public class Balancer {
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

@ -75,7 +75,7 @@ public interface BlockCollection {
* and set the locations.
*/
public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeDescriptor[] locations) throws IOException;
DatanodeStorageInfo[] targets) throws IOException;
/**
* @return whether the block collection is under construction.

View File

@ -21,6 +21,7 @@ import java.util.LinkedList;
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;
@ -39,11 +40,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
@ -86,9 +87,14 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
}
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) {
@ -111,14 +117,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
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;
}
/**
@ -190,22 +192,34 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
}
/**
* 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 :
@ -213,10 +227,13 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
// 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;
}
@ -236,37 +253,70 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
}
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;
@ -275,9 +325,9 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
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;
@ -289,7 +339,7 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
*
* @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;
@ -298,9 +348,9 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
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;
}
@ -328,10 +378,10 @@ public class BlockInfo extends Block implements LightWeightGSet.LinkedElement {
* @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 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* 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 @@ public class BlockInfoUnderConstruction extends BlockInfo {
* Is data-node the replica belongs to alive.
*/
boolean isAlive() {
return expectedLocation.isAlive;
return expectedLocation.getDatanodeDescriptor().isAlive;
}
@Override // Block
@ -162,7 +162,7 @@ public class BlockInfoUnderConstruction extends BlockInfo {
*/
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 @@ public class BlockInfoUnderConstruction extends BlockInfo {
}
/** 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 class BlockInfoUnderConstruction extends BlockInfo {
* 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 class BlockInfoUnderConstruction extends BlockInfo {
// 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 class BlockInfoUnderConstruction extends BlockInfo {
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.Map;
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.conf.Configuration;
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.BlockCommand;
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 class BlockManager {
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 @@ public class BlockManager {
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 @@ public class BlockManager {
state = "(decommissioned)";
}
if (node.areBlockContentsStale()) {
if (storage.areBlockContentsStale()) {
state += " (block deletions maybe out of date)";
}
out.print(" " + node + state + " : ");
@ -660,10 +663,9 @@ public class BlockManager {
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 class BlockManager {
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 class BlockManager {
/**
* 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 @@ public class BlockManager {
+ ", 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 @@ public class BlockManager {
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 @@ public class BlockManager {
}
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 @@ public class BlockManager {
*/
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 @@ public class BlockManager {
* 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 class BlockManager {
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 @@ public class BlockManager {
}
// 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 @@ public class BlockManager {
@VisibleForTesting
int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
int requiredReplication, numEffectiveReplicas;
List<DatanodeDescriptor> containingNodes, liveReplicaNodes;
List<DatanodeDescriptor> containingNodes;
DatanodeDescriptor srcNode;
BlockCollection bc = null;
int additionalReplRequired;
@ -1218,7 +1223,7 @@ public class BlockManager {
// 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,
@ -1277,7 +1282,7 @@ public class BlockManager {
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;
@ -1315,7 +1320,8 @@ public class BlockManager {
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;
}
@ -1324,15 +1330,13 @@ public class BlockManager {
// 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
@ -1352,12 +1356,12 @@ public class BlockManager {
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);
@ -1381,15 +1385,16 @@ public class BlockManager {
* @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 (="
@ -1450,12 +1455,11 @@ public class BlockManager {
* 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;
@ -1463,12 +1467,12 @@ public class BlockManager {
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())
@ -1476,7 +1480,7 @@ public class BlockManager {
else if (excessBlocks != null && excessBlocks.contains(block)) {
excess++;
} else {
nodesContainingLiveReplicas.add(node);
nodesContainingLiveReplicas.add(storage);
live++;
}
containingNodes.add(node);
@ -1608,10 +1612,11 @@ public class BlockManager {
}
/**
* 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
@ -1625,26 +1630,28 @@ public class BlockManager {
// 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");
@ -1698,28 +1705,30 @@ public class BlockManager {
}
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) {
@ -1733,7 +1742,7 @@ public class BlockManager {
addToInvalidates(b, node);
}
for (BlockToMarkCorrupt b : toCorrupt) {
markBlockAsCorrupt(b, node);
markBlockAsCorrupt(b, node, storage.getStorageID());
}
}
@ -1749,10 +1758,11 @@ public class BlockManager {
* @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()) {
@ -1761,7 +1771,7 @@ public class BlockManager {
if (shouldPostponeBlocksFromFuture &&
namesystem.isGenStampInFuture(iblk)) {
queueReportedBlock(node, iblk, reportedState,
queueReportedBlock(node, storageID, iblk, reportedState,
QUEUE_REASON_FUTURE_GENSTAMP);
continue;
}
@ -1778,10 +1788,10 @@ public class BlockManager {
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;
}
@ -1789,7 +1799,7 @@ public class BlockManager {
// 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
@ -1802,22 +1812,25 @@ public class BlockManager {
}
//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;
@ -1829,20 +1842,21 @@ public class BlockManager {
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);
}
/**
@ -1876,7 +1890,8 @@ public class BlockManager {
* @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,
@ -1891,7 +1906,7 @@ public class BlockManager {
if (shouldPostponeBlocksFromFuture &&
namesystem.isGenStampInFuture(block)) {
queueReportedBlock(dn, block, reportedState,
queueReportedBlock(dn, storageID, block, reportedState,
QUEUE_REASON_FUTURE_GENSTAMP);
return null;
}
@ -1912,7 +1927,7 @@ public class BlockManager {
}
// 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; */
@ -1926,7 +1941,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
// 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);
@ -1955,7 +1970,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
* 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;
@ -1965,7 +1980,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
" from datanode " + dn + " for later processing " +
"because " + reason + ".");
}
pendingDNMessages.enqueueReportedBlock(dn, block, reportedState);
pendingDNMessages.enqueueReportedBlock(dn, storageID, block, reportedState);
}
/**
@ -1988,8 +2003,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
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);
}
}
@ -2106,19 +2121,21 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
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
@ -2129,17 +2146,17 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
* @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);
@ -2162,6 +2179,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
*/
private Block addStoredBlock(final BlockInfo block,
DatanodeDescriptor node,
String storageID,
DatanodeDescriptor delNodeHint,
boolean logEveryBlock)
throws IOException {
@ -2187,7 +2205,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + 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) {
@ -2447,19 +2465,19 @@ assert storedBlock.findDatanode(dn) < 0 : "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
@ -2548,10 +2566,10 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
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();
@ -2599,7 +2617,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
// in "excess" there.
//
LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(node
.getStorageID());
.getDatanodeUuid());
if (excessBlocks != null) {
if (excessBlocks.remove(block)) {
excessBlocksCount.decrementAndGet();
@ -2608,7 +2626,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
+ block + " is removed from excessBlocks");
}
if (excessBlocks.size() == 0) {
excessReplicateMap.remove(node.getStorageID());
excessReplicateMap.remove(node.getDatanodeUuid());
}
}
}
@ -2623,12 +2641,18 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
* 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();
}
}
@ -2637,12 +2661,12 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
* 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;
@ -2658,11 +2682,12 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
// 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
@ -2670,7 +2695,7 @@ assert storedBlock.findDatanode(dn) < 0 : "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
@ -2678,11 +2703,11 @@ assert storedBlock.findDatanode(dn) < 0 : "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) {
@ -2696,7 +2721,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
addToInvalidates(b, node);
}
for (BlockToMarkCorrupt b : toCorrupt) {
markBlockAsCorrupt(b, node);
markBlockAsCorrupt(b, node, storageID);
}
}
@ -2708,7 +2733,7 @@ assert storedBlock.findDatanode(dn) < 0 : "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;
@ -2724,19 +2749,19 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
"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:
@ -2768,24 +2793,23 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
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++;
}
}
@ -2808,10 +2832,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
}
// 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++;
}
@ -2823,10 +2846,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
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(" ");
}
@ -2923,14 +2945,13 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
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() {
@ -3059,9 +3080,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
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 ||
@ -3105,8 +3125,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
}
/** @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) {
@ -3257,24 +3277,24 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
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;
@ -3283,8 +3303,8 @@ assert storedBlock.findDatanode(dn) < 0 : "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.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.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 abstract class BlockPlacementPolicy {
* @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 class BlockPlacementPolicy {
* 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 java.util.TreeSet;
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 class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
}
@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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
// 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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
* 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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
* 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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
}
/**
* 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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
* 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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
// 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 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
* 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.Map;
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;
@ -64,81 +66,87 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
* @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;
}
@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);
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);
}
}
}
@ -146,8 +154,9 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
@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(
@ -155,12 +164,12 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
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);
}
}
@ -170,46 +179,40 @@ public class BlockPlacementPolicyWithNodeGroup extends BlockPlacementPolicyDefau
* 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);
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);
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);
}
}
}

View File

@ -29,11 +29,11 @@ import org.apache.hadoop.util.LightWeightGSet;
* 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 @@ class BlocksMap {
}
@Override
public DatanodeDescriptor next() {
return blockInfo.getDatanode(nextIdx++);
public DatanodeStorageInfo next() {
return blockInfo.getStorageInfo(nextIdx++);
}
@Override
@ -115,18 +115,23 @@ class BlocksMap {
/**
* 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

@ -18,23 +18,29 @@
package org.apache.hadoop.hdfs.server.blockmanagement;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
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.namenode.CachedBlock;
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.IntrusiveCollection;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
/**
* This class extends the DatanodeInfo class with ephemeral information (eg
* health, capacity, what blocks are associated with the Datanode) that is
@ -43,6 +49,7 @@ import com.google.common.annotations.VisibleForTesting;
@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.
@ -54,9 +61,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;
}
@ -99,6 +106,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
}
}
private final Map<String, DatanodeStorageInfo> storageMap =
new HashMap<String, DatanodeStorageInfo>();
/**
* A list of CachedBlock objects on this datanode.
*/
@ -164,37 +174,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
*/
private long lastCachingDirectiveSentTimeMs;
/**
* Head of the list of blocks on the datanode
*/
private volatile BlockInfo blockList = null;
/**
* Number of blocks on the datanode
*/
private int numBlocks = 0;
// isAlive == heartbeats.contains(this)
// This is an optimization, because contains takes O(n) time on Arraylist
public boolean isAlive = false;
public boolean needKeyUpdate = false;
/**
* 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
@ -213,7 +197,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
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).
*/
@ -223,9 +207,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
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
@ -237,7 +218,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
* @param nodeID id of the data node
*/
public DatanodeDescriptor(DatanodeID nodeID) {
this(nodeID, 0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
super(nodeID);
updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
}
/**
@ -247,104 +229,60 @@ public class DatanodeDescriptor extends DatanodeInfo {
*/
public DatanodeDescriptor(DatanodeID nodeID,
String networkLocation) {
this(nodeID, networkLocation, 0L, 0L, 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 cacheCapacity cache capacity of the data node
* @param cacheUsed cache used on the data node
* @param xceiverCount # of data transfers at the data node
*/
public DatanodeDescriptor(DatanodeID nodeID,
long capacity,
long dfsUsed,
long remaining,
long bpused,
long cacheCapacity,
long cacheUsed,
int xceiverCount,
int failedVolumes) {
super(nodeID);
updateHeartbeat(capacity, dfsUsed, remaining, bpused, cacheCapacity,
cacheUsed, 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 cacheCapacity cache capacity of the data node
* @param cacheUsed cache used on the data node
* @param xceiverCount # of data transfers at the data node
*/
public DatanodeDescriptor(DatanodeID nodeID,
String networkLocation,
long capacity,
long dfsUsed,
long remaining,
long bpused,
long cacheCapacity,
long cacheUsed,
int xceiverCount,
int failedVolumes) {
super(nodeID, networkLocation);
updateHeartbeat(capacity, dfsUsed, remaining, bpused, cacheCapacity,
cacheUsed, xceiverCount, failedVolumes);
updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 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.
*/
@VisibleForTesting
protected BlockInfo getHead(){
return blockList;
boolean removeBlock(String storageID, BlockInfo b) {
DatanodeStorageInfo s = getStorageInfo(storageID);
if (s != null) {
return s.removeBlock(b);
}
return false;
}
/**
@ -355,9 +293,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
* @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;
}
@ -368,7 +309,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
setBlockPoolUsed(0);
setDfsUsed(0);
setXceiverCount(0);
this.blockList = null;
this.invalidateBlocks.clear();
this.volumeFailures = 0;
// pendingCached, cached, and pendingUncached are protected by the
@ -392,66 +332,97 @@ public class DatanodeDescriptor extends DatanodeInfo {
}
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, long cacheCapacity, long cacheUsed, int xceiverCount,
int volFailures) {
setCapacity(capacity);
setRemaining(remaining);
setBlockPoolUsed(blockPoolUsed);
setDfsUsed(dfsUsed);
public void updateHeartbeat(StorageReport[] reports, long cacheCapacity,
long cacheUsed, int xceiverCount, int volFailures) {
long totalCapacity = 0;
long totalRemaining = 0;
long totalBlockPoolUsed = 0;
long totalDfsUsed = 0;
setCacheCapacity(cacheCapacity);
setCacheUsed(cacheUsed);
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));
}
@ -526,18 +497,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
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) {
@ -546,12 +513,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
// 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;
@ -647,7 +611,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
@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);
}
}
/**
@ -664,26 +632,6 @@ public class DatanodeDescriptor extends DatanodeInfo {
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());
@ -702,6 +650,19 @@ public class DatanodeDescriptor extends DatanodeInfo {
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;
}
}
/**
* @return The time at which we last sent caching directives to this
* DataNode, in monotonic milliseconds.
@ -718,3 +679,4 @@ public class DatanodeDescriptor extends DatanodeInfo {
this.lastCachingDirectiveSentTimeMs = time;
}
}

View File

@ -424,9 +424,13 @@ public class DatanodeManager {
}
/** 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);
}
/**
@ -438,7 +442,7 @@ public class DatanodeManager {
*/
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())) {
@ -451,6 +455,20 @@ public class DatanodeManager {
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) {
@ -528,7 +546,7 @@ public class DatanodeManager {
// 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
@ -543,7 +561,7 @@ public class DatanodeManager {
/** 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));
}
@ -705,8 +723,10 @@ public class DatanodeManager {
/** 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());
@ -728,24 +748,6 @@ public class DatanodeManager {
}
}
/**
* 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.
@ -784,9 +786,9 @@ public class DatanodeManager {
}
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());
@ -821,7 +823,7 @@ public class DatanodeManager {
*/
NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS
+ " is replaced by " + nodeReg + " with the same storageID "
+ nodeReg.getStorageID());
+ nodeReg.getDatanodeUuid());
}
boolean success = false;
@ -853,20 +855,8 @@ public class DatanodeManager {
}
}
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;
@ -1234,10 +1224,10 @@ public class DatanodeManager {
/** Handle heartbeat from datanodes. */
public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg,
final String blockPoolId,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
long cacheCapacity, long cacheUsed, int xceiverCount, int maxTransfers,
int failedVolumes) throws IOException {
StorageReport[] reports, final String blockPoolId,
long cacheCapacity, long cacheUsed, int xceiverCount,
int maxTransfers, int failedVolumes
) throws IOException {
synchronized (heartbeatManager) {
synchronized (datanodeMap) {
DatanodeDescriptor nodeinfo = null;
@ -1257,9 +1247,9 @@ public class DatanodeManager {
return new DatanodeCommand[]{RegisterCommand.REGISTER};
}
heartbeatManager.updateHeartbeat(nodeinfo, capacity, dfsUsed,
remaining, blockPoolUsed, cacheCapacity, cacheUsed, xceiverCount,
failedVolumes);
heartbeatManager.updateHeartbeat(nodeinfo, reports,
cacheCapacity, cacheUsed,
xceiverCount, failedVolumes);
// If we are in safemode, do not send back any recovery / replication
// requests. Don't even drain the existing queue of work.
@ -1274,32 +1264,32 @@ public class DatanodeManager {
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()));
}
}
@ -1416,7 +1406,9 @@ public class DatanodeManager {
LOG.info("Marking all datandoes as stale");
synchronized (datanodeMap) {
for (DatanodeDescriptor dn : datanodeMap.values()) {
dn.markStaleAfterFailover();
for(DatanodeStorageInfo storage : dn.getStorageInfos()) {
storage.markStaleAfterFailover();
}
}
}
}
@ -1455,3 +1447,4 @@ public class DatanodeManager {
this.shouldSendCachingCommands = shouldSendCachingCommands;
}
}

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.DFSConfigKeys;
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;
@ -181,7 +182,7 @@ class HeartbeatManager implements DatanodeStatistics {
addDatanode(d);
//update its timestamp
d.updateHeartbeat(0L, 0L, 0L, 0L, 0L, 0L, 0, 0);
d.updateHeartbeat(StorageReport.EMPTY_ARRAY, 0L, 0L, 0, 0);
}
}
@ -203,11 +204,11 @@ class HeartbeatManager implements DatanodeStatistics {
}
synchronized void updateHeartbeat(final DatanodeDescriptor node,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
long cacheCapacity, long cacheUsed, int xceiverCount, int failedVolumes) {
StorageReport[] reports, long cacheCapacity, long cacheUsed,
int xceiverCount, int failedVolumes) {
stats.subtract(node);
node.updateHeartbeat(capacity, dfsUsed, remaining, blockPoolUsed,
cacheCapacity, cacheUsed, xceiverCount, failedVolumes);
node.updateHeartbeat(reports, cacheCapacity, cacheUsed,
xceiverCount, failedVolumes);
stats.add(node);
}
@ -358,3 +359,4 @@ class HeartbeatManager implements DatanodeStatistics {
}
}
}

View File

@ -78,10 +78,10 @@ class InvalidateBlocks {
*/
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 @@ class PendingDataNodeMessages {
DatanodeDescriptor getNode() {
return dn;
}
String getStorageID() {
return storageID;
}
ReplicaState getReportedState() {
return reportedState;
@ -70,11 +76,11 @@ class PendingDataNodeMessages {
}
}
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 abstract class Storage extends StorageInfo {
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 abstract class Storage extends StorageInfo {
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 java.util.concurrent.CopyOnWriteArrayList;
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;
@ -160,31 +161,32 @@ class BPOfferService {
synchronized NamespaceInfo getNamespaceInfo() {
return bpNSInfo;
}
@Override
public 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);
}
}
@ -193,7 +195,8 @@ class BPOfferService {
* 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(
@ -202,7 +205,7 @@ class BPOfferService {
delHint);
for (BPServiceActor actor : bpServices) {
actor.notifyNamenodeBlockImmediately(bInfo);
actor.notifyNamenodeBlockImmediately(bInfo, storageUuid);
}
}
@ -219,23 +222,23 @@ class BPOfferService {
"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);
}
}
@ -338,7 +341,7 @@ class BPOfferService {
}
}
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 static org.apache.hadoop.util.Time.now;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.SocketTimeoutException;
import java.net.URI;
import java.util.Collection;
import java.util.List;
import java.util.Map;
@ -31,6 +30,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.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;
@ -52,7 +52,6 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
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;
@ -100,9 +99,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;
@ -244,12 +243,15 @@ class BPServiceActor implements Runnable {
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);
@ -263,49 +265,100 @@ class BPServiceActor implements Runnable {
}
/**
* 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);
}
/*
@ -313,19 +366,19 @@ class BPServiceActor implements Runnable {
* 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);
}
}
@ -334,13 +387,13 @@ class BPServiceActor implements Runnable {
*/
@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;
}
@ -350,12 +403,12 @@ class BPServiceActor implements Runnable {
@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;
}
@ -365,13 +418,13 @@ class BPServiceActor implements Runnable {
@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;
}
@ -395,23 +448,38 @@ class BPServiceActor implements Runnable {
// 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");
@ -466,17 +534,15 @@ class BPServiceActor implements Runnable {
}
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.getFSDataset().getCacheCapacity(),
dn.getFSDataset().getCacheUsed(),
dn.getXmitsInProgress(),
@ -496,9 +562,9 @@ class BPServiceActor implements Runnable {
}
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;
}
@ -608,10 +674,10 @@ class BPServiceActor implements Runnable {
//
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");
}
@ -782,4 +848,68 @@ class BPServiceActor implements Runnable {
}
}
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 @@ class BlockPoolSliceScanner {
+ 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.
*/
@ -1073,14 +1080,15 @@ class BlockReceiver implements Closeable {
: 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 = datanode.getDNRegistrationForBP(block
.getBlockPoolId());
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;
@ -38,21 +68,42 @@ import org.apache.hadoop.hdfs.HDFSPolicyProvider;
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;
@ -61,7 +112,11 @@ import org.apache.hadoop.hdfs.server.datanode.web.resources.DatanodeWebHdfsMetho
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.UserGroupInformation.AuthenticationMethod;
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 class DataNode extends Configured
private JvmPauseMonitor pauseMonitor;
private SecureResources secureResources = null;
private AbstractList<File> dataDirs;
private List<StorageLocation> dataDirs;
private Configuration conf;
private final long maxNumberOfBlocksToLog;
@ -219,21 +272,12 @@ public class DataNode extends Configured
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);
this.maxNumberOfBlocksToLog = conf.getLong(DFS_MAX_NUM_BLOCKS_TO_LOG_KEY,
@ -494,7 +538,7 @@ public class DataNode extends Configured
directoryScanner.start();
} else {
LOG.info("Periodic Directory Tree Verification scan is disabled because " +
reason);
reason);
}
}
@ -566,10 +610,11 @@ public class DataNode extends Configured
}
// 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());
@ -577,10 +622,11 @@ public class DataNode extends Configured
}
// 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());
@ -588,10 +634,10 @@ public class DataNode extends Configured
}
/** 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());
@ -603,7 +649,9 @@ public class DataNode extends Configured
*/
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());
}
/**
@ -675,7 +723,7 @@ public class DataNode extends Configured
* @throws IOException
*/
void startDataNode(Configuration conf,
AbstractList<File> dataDirs,
List<StorageLocation> dataDirs,
// DatanodeProtocol namenode,
SecureResources resources
) throws IOException {
@ -736,19 +784,42 @@ public class DataNode extends Configured
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());
@ -767,16 +838,10 @@ public class DataNode extends Configured
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);
@ -897,7 +962,7 @@ public class DataNode extends Configured
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) {
@ -924,10 +989,6 @@ public class DataNode extends Configured
return streamingAddr.getPort();
}
String getStorageId() {
return storage.getStorageID();
}
/**
* @return name useful for logging
*/
@ -1013,34 +1074,6 @@ public class DataNode extends Configured
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
@ -1370,8 +1403,10 @@ public class DataNode extends Configured
// 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());
@ -1635,11 +1670,11 @@ public class DataNode extends Configured
* @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());
@ -1703,17 +1738,32 @@ public class DataNode extends Configured
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.
@ -1779,57 +1829,52 @@ public class DataNode extends Configured
* 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() + "}";
}
@ -1883,7 +1928,6 @@ public class DataNode extends Configured
}
/**
* 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.
*
@ -1983,7 +2027,7 @@ public class DataNode extends Configured
ExtendedBlock newBlock = new ExtendedBlock(oldBlock);
newBlock.setGenerationStamp(recoveryId);
newBlock.setNumBytes(newLength);
notifyNamenodeReceivedBlock(newBlock, "");
notifyNamenodeReceivedBlock(newBlock, "", storageID);
return storageID;
}
@ -2443,6 +2487,10 @@ public class DataNode extends Configured
return dnConf;
}
public String getDatanodeUuid() {
return id == null ? null : id.getDatanodeUuid();
}
boolean shouldRun() {
return shouldRun;
}

View File

@ -24,13 +24,7 @@ import java.io.FileOutputStream;
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.HdfsServerConstants.StartupOption;
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 class DataStorage extends Storage {
* @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 @@ public class DataStorage extends Storage {
// 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 @@ public class DataStorage extends Storage {
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 @@ public class DataStorage extends Storage {
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 @@ public class DataStorage extends Storage {
* @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 @@ public class DataStorage extends Storage {
}
}
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 @@ public class DataStorage extends Storage {
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 @@ public class DataStorage extends Storage {
/*
* 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 @@ public class DataStorage extends Storage {
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 @@ class DataXceiver extends Receiver implements Runnable {
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 @@ class DataXceiver extends Receiver implements Runnable {
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 @@ class DataXceiver extends Receiver implements Runnable {
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 @@ class DataXceiver extends Receiver implements Runnable {
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 @@ class DataXceiver extends Receiver implements Runnable {
// 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 @@ class DataXceiver extends Receiver implements Runnable {
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 @@ package org.apache.hadoop.hdfs.server.datanode;
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 @@ public class DirectoryScanner implements Runnable {
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 @@ public class DirectoryScanner implements Runnable {
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 @@ public class DirectoryScanner implements Runnable {
}
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 @@ public class DirectoryScanner implements Runnable {
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 @@ public class DirectoryScanner implements Runnable {
/** 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 @@ abstract public class ReplicaInfo extends Block implements Replica {
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.ExtendedBlock;
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 interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
/** @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 interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
* @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 interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
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);
/**
* Returns the cache report - the full list of cached block IDs of a
@ -408,3 +412,4 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
public HdfsBlocksMetadata getHdfsBlocksMetadata(List<ExtendedBlock> blocks)
throws IOException;
}

View File

@ -20,10 +20,15 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset;
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 @@ class FsDatasetAsyncDiskService {
+ " 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

@ -38,13 +38,14 @@ import javax.management.NotCompliantMBeanException;
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;
@ -53,6 +54,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
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;
@ -66,6 +68,7 @@ import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
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;
@ -76,7 +79,9 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosing
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;
@ -108,6 +113,26 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
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());
@ -169,11 +194,12 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
final DataNode datanode;
final FsVolumeList volumes;
final ReplicaMap volumeMap;
final FsDatasetAsyncDiskService asyncDiskService;
final FsDatasetCache cacheManager;
private final int validVolsRequired;
final ReplicaMap volumeMap;
// Used for synchronizing access to usage stats
private final Object statsLock = new Object();
@ -190,6 +216,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
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();
@ -210,9 +237,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
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);
@ -223,7 +256,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
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++) {
@ -231,7 +264,17 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
}
asyncDiskService = new FsDatasetAsyncDiskService(datanode, roots);
cacheManager = new FsDatasetCache(this);
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;
}
/**
@ -336,9 +379,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
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;
@ -684,7 +724,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
}
@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
@ -695,6 +735,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
if (replicaInfo.getState() == ReplicaState.RBW) {
finalizeReplica(b.getBlockPoolId(), replicaInfo);
}
return replicaInfo.getStorageUuid();
}
/**
@ -995,56 +1036,68 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
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);
}
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.";
}
}
return new BlockListAsLongs(finalized, uc);
}
}
@Override // FsDatasetSpi
public List<Long> getCacheReport(String bpid) {
return cacheManager.getCachedBlocks(bpid);
}
@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.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.";
}
}
}
for (FsVolumeImpl v : volumes.volumes) {
ArrayList<ReplicaInfo> finalizedList = finalized.get(v.getStorageID());
ArrayList<ReplicaInfo> ucList = uc.get(v.getStorageID());
blockReportsMap.put(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;
@ -1333,22 +1386,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
/**
* 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);
}
@ -1724,7 +1770,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
LOG.info("Adding block pool " + bpid);
volumes.addBlockPool(bpid, conf);
volumeMap.initBlockPool(bpid);
volumes.getVolumeMap(bpid, volumeMap);
volumes.getAllVolumesMap(bpid, volumeMap);
}
@Override
@ -1734,11 +1780,6 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
volumes.removeBlockPool(bpid);
}
@Override
public String[] getBlockPoolList() {
return volumeMap.getBlockPoolList();
}
/**
* Class for representing the Datanode volume information
*/
@ -1871,3 +1912,4 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
return new RollingLogsImpl(dir, prefix);
}
}

View File

@ -34,9 +34,11 @@ import org.apache.hadoop.conf.Configuration;
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;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
@ -50,6 +52,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
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
@ -64,7 +67,7 @@ class FsVolumeImpl implements FsVolumeSpi {
private final ThreadPoolExecutor cacheExecutor;
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(
@ -73,6 +76,7 @@ class FsVolumeImpl implements FsVolumeSpi {
this.currentDir = currentDir;
File parent = currentDir.getParentFile();
this.usage = new DF(parent, conf);
this.storageType = storageType;
final int maxNumThreads = dataset.datanode.getConf().getInt(
DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY,
DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_DEFAULT
@ -320,7 +324,19 @@ class FsVolumeImpl implements FsVolumeSpi {
}
}
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 @@ class FsVolumeList {
* @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 @@ class FsVolumeList {
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 @@ class ReplicaMap {
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

@ -62,6 +62,7 @@ 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.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;
@ -364,7 +365,7 @@ public class FSDirectory implements Closeable {
* 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.DFSConfigKeys;
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.CacheDirectiveEntry;
@ -174,14 +175,7 @@ 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.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;
@ -218,7 +212,8 @@ import org.apache.hadoop.hdfs.server.protocol.NNHAStatusHeartbeat;
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;
@ -2582,7 +2577,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
// 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.
@ -2709,7 +2704,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
src + ". Returning previously allocated block " + lastBlockInFile);
long offset = pendingFile.computeFileSize();
onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
((BlockInfoUnderConstruction)lastBlockInFile).getExpectedLocations(),
((BlockInfoUnderConstruction)lastBlockInFile).getExpectedStorageLocations(),
offset);
return iip;
} else {
@ -2727,11 +2722,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
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;
@ -2739,7 +2733,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
/** @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
@ -2747,7 +2742,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
final DatanodeDescriptor clientnode;
final long preferredblocksize;
final List<DatanodeDescriptor> chosen;
final List<DatanodeStorageInfo> chosen;
checkOperation(OperationCategory.READ);
byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
readLock();
@ -2762,23 +2757,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
clientnode = file.getFileUnderConstructionFeature().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;
@ -2949,14 +2939,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* @throws QuotaExceededException If addition of block exceeds space quota
*/
BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
Block newBlock, DatanodeDescriptor targets[]) throws IOException {
Block newBlock, DatanodeStorageInfo[] targets)
throws IOException {
assert hasWriteLock();
BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets);
NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
+ getBlockPoolId() + " " + b);
for (DatanodeDescriptor dn : targets) {
dn.incBlocksScheduled();
}
DatanodeStorageInfo.incrementBlocksScheduled(targets);
return b;
}
@ -3419,7 +3408,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
boolean isFileClosed(String src)
throws AccessControlException, UnresolvedLinkException,
StandbyException, IOException {
FSPermissionChecker pc = getPermissionChecker();
FSPermissionChecker pc = getPermissionChecker();
checkOperation(OperationCategory.READ);
readLock();
try {
@ -3716,7 +3705,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
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) {
@ -3915,32 +3904,39 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
// 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()];
iFile.setLastBlock(storedBlock, targetList.toArray(targetArray));
DatanodeStorageInfo[] trimmedStorageInfos =
blockManager.getDatanodeManager().getDatanodeStorageInfos(
trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]),
trimmedStorages.toArray(new String[trimmedStorages.size()]));
iFile.setLastBlock(storedBlock, trimmedStorageInfos);
}
if (closeFile) {
@ -4142,16 +4138,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* @throws IOException
*/
HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
long cacheCapacity, long cacheUsed, int xceiverCount, int xmitsInProgress,
int failedVolumes) throws IOException {
StorageReport[] reports, long cacheCapacity, long cacheUsed,
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,
cacheCapacity, cacheUsed, xceiverCount, maxTransfer, failedVolumes);
nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
xceiverCount, maxTransfer, failedVolumes);
return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
} finally {
readUnlock();
@ -5312,11 +5308,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
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();
}
@ -5805,9 +5801,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
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");
}
}
@ -5862,7 +5859,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* @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);
@ -5883,7 +5880,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
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();
@ -5895,7 +5892,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
/** @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
@ -5918,15 +5916,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
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);
@ -6053,7 +6045,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
* @throws IOException
*/
Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
String[] cookieTab) throws IOException {
String[] cookieTab) throws IOException {
checkSuperuserPrivilege();
checkOperation(OperationCategory.READ);
readLock();
@ -7374,3 +7366,4 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
}
}
}

View File

@ -27,10 +27,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
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.BlockCollection;
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.*;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
@ -234,7 +231,7 @@ public class INodeFile extends INodeWithAdditionalFields
@Override // BlockCollection, the file should be under construction
public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
DatanodeDescriptor[] locations) throws IOException {
DatanodeStorageInfo[] locations) throws IOException {
Preconditions.checkState(isUnderConstruction(),
"file is no longer under construction");

View File

@ -43,8 +43,8 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
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.BatchedRemoteIterator.BatchedEntries;
import org.apache.hadoop.fs.permission.FsPermission;
@ -65,21 +65,21 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
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;
@ -95,9 +95,9 @@ import org.apache.hadoop.hdfs.protocolPB.NamenodeProtocolServerSideTranslatorPB;
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;
@ -198,9 +198,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
NamenodeProtocolServerSideTranslatorPB namenodeProtocolXlator =
new NamenodeProtocolServerSideTranslatorPB(this);
BlockingService NNPbService = NamenodeProtocolService
BlockingService NNPbService = NamenodeProtocolService
.newReflectiveBlockingService(namenodeProtocolXlator);
RefreshAuthorizationPolicyProtocolServerSideTranslatorPB refreshAuthPolicyXlator =
new RefreshAuthorizationPolicyProtocolServerSideTranslatorPB(this);
BlockingService refreshAuthService = RefreshAuthorizationPolicyProtocolService
@ -220,7 +220,7 @@ class NameNodeRpcServer implements NamenodeProtocols {
new HAServiceProtocolServerSideTranslatorPB(this);
BlockingService haPbService = HAServiceProtocolService
.newReflectiveBlockingService(haServiceProtocolXlator);
WritableRpcEngine.ensureInitialized();
InetSocketAddress serviceRpcAddr = nn.getServiceRpcServerAddress(conf);
@ -571,7 +571,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
@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()) {
@ -592,8 +593,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
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.
@ -641,9 +642,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
@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
@ -965,24 +966,25 @@ class NameNodeRpcServer implements NamenodeProtocols {
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(), dnCacheCapacity, dnCacheUsed,
xceiverCount, xmitsInProgress, failedVolumes);
return namesystem.handleHeartbeat(nodeReg, report,
dnCacheCapacity, dnCacheUsed, 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;
@ -1009,8 +1011,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
+"from "+nodeReg+" "+receivedAndDeletedBlocks.length
+" blocks.");
}
namesystem.processIncrementalBlockReport(
nodeReg, poolId, receivedAndDeletedBlocks[0].getBlocks());
for(StorageReceivedDeletedBlocks r : receivedAndDeletedBlocks) {
namesystem.processIncrementalBlockReport(nodeReg, poolId, r);
}
}
@Override // DatanodeProtocol
@ -1280,3 +1283,4 @@ class NameNodeRpcServer implements NamenodeProtocols {
return namesystem.listCachePools(prevKey != null ? prevKey : "");
}
}

View File

@ -27,13 +27,7 @@ import java.net.InetSocketAddress;
import java.net.URI;
import java.net.URLEncoder;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.*;
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServletRequest;
@ -50,6 +44,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
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;
@ -1113,13 +1108,12 @@ class NamenodeJspHelper {
}
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.conf.Configuration;
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.DelegationTokenIdentifie
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 @@ public class NamenodeWebHdfsMethods {
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 @@ public class NamenodeWebHdfsMethods {
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.classification.InterfaceStability;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
import org.apache.hadoop.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 class BlockCommand extends DatanodeCommand {
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 class BlockCommand extends DatanodeCommand {
* @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 class BlockCommand extends DatanodeCommand {
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 class BlocksWithLocations {
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 class DatanodeRegistration extends DatanodeID
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 class DatanodeStorage {
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 class DatanodeStorage {
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, -48 };
-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 @@ public class JsonUtil {
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());
@ -329,6 +330,7 @@ public class JsonUtil {
return null;
}
// TODO: Fix storageID
return new DatanodeInfo(
(String)m.get("ipAddr"),
(String)m.get("hostName"),
@ -412,7 +414,7 @@ public class JsonUtil {
(Object[])m.get("cachedLocations"));
final LocatedBlock locatedblock = new LocatedBlock(b, locations,
startOffset, isCorrupt, cachedLocations);
null, null, startOffset, isCorrupt, cachedLocations);
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 {
@ -545,6 +546,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

@ -53,8 +53,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];
}
/**
@ -106,10 +107,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;
}
/**
@ -193,7 +196,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 ];
@ -284,7 +287,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
@ -116,6 +119,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.
@ -130,6 +147,8 @@ message LocatedBlockProto {
required hadoop.common.TokenProto blockToken = 5;
repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
repeated StorageTypeProto storageTypes = 7;
repeated string storageIDs = 8;
}
message DataEncryptionKeyProto {
@ -336,7 +355,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
}
/**
@ -439,3 +459,4 @@ message SnapshotInfoProto {
// TODO: do we need access time?
}

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.fs.*;
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.security.token.block.ExportedBlockKeys;
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.datanode.TestTransferRbw;
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;
@ -770,7 +773,8 @@ public class DFSTestUtil {
}
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,
@ -782,7 +786,8 @@ public class DFSTestUtil {
}
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);
}
@ -804,8 +809,9 @@ public class DFSTestUtil {
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));
}
@ -826,9 +832,43 @@ public class DFSTestUtil {
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.security.PrivilegedExceptionAction;
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.datanode.fsdataset.FsDatasetSpi;
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 @@ public class MiniDFSCluster {
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,
@ -1148,15 +1154,16 @@ public class MiniDFSCluster {
// 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);
}
@ -1926,12 +1933,14 @@ public class MiniDFSCluster {
// 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;
}
}
@ -1940,6 +1949,7 @@ public class MiniDFSCluster {
// using (capacity == 0) as proxy.
for (DatanodeInfo dn : dnInfo) {
if (dn.getCapacity() == 0) {
LOG.info("dn.getCapacity() == 0");
return true;
}
}
@ -1947,6 +1957,7 @@ public class MiniDFSCluster {
// 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;
}
}
@ -1966,12 +1977,12 @@ public class MiniDFSCluster {
* @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);
}
@ -1980,11 +1991,12 @@ public class MiniDFSCluster {
* @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.security.PrivilegedExceptionAction;
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.conf.Configuration;
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;
@ -1392,11 +1395,14 @@ public class TestDFSShell {
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 @@ public class TestDFSStartupVersions {
* 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,7 @@ package org.apache.hadoop.hdfs;
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;
@ -82,7 +83,8 @@ public class TestDatanodeConfig {
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

@ -173,7 +173,7 @@ public class TestDatanodeRegistration {
// 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();
@ -190,7 +190,7 @@ public class TestDatanodeRegistration {
// 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());
@ -226,7 +226,7 @@ public class TestDatanodeRegistration {
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.
@ -273,7 +273,7 @@ public class TestDatanodeRegistration {
DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
doReturn(HdfsConstants.LAYOUT_VERSION).when(mockDnReg).getVersion();
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 class TestFileCorruption {
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,20 +22,21 @@ import static org.junit.Assert.assertEquals;
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;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.FSDataOutputStream;
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;
@ -136,7 +137,7 @@ public class TestInjectionForSimulatedStorage {
DFSTestUtil.createFile(cluster.getFileSystem(), testPath, filesize,
filesize, blockSize, (short) numDataNodes, 0L);
waitForBlockReplication(testFile, dfsClient.getNamenode(), numDataNodes, 20);
Iterable<Block>[] blocksList = cluster.getAllBlockReports(bpid);
List<Map<DatanodeStorage, BlockListAsLongs>> blocksList = cluster.getAllBlockReports(bpid);
cluster.shutdown();
cluster = null;
@ -157,9 +158,11 @@ public class TestInjectionForSimulatedStorage {
.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 class TestPeerCache {
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 class TestPeerCache {
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 class TestPeerCache {
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 class TestPeerCache {
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 class TestPeerCache {
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 class UpgradeUtilities {
*/
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.HdfsProtos;
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.security.token.block.ExportedBlockKeys;
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 class TestPBHelper {
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 class TestPBHelper {
}
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 @@ public class TestPBHelper {
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, new DatanodeInfo[]{});
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 class TestPBHelper {
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 class TestPBHelper {
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 class TestPBHelper {
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 java.util.concurrent.TimeoutException;
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.balancer.Balancer.Cli;
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 @@ import org.junit.Test;
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 @@ public class TestBalancer {
} 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 @@ public class TestBalancer {
*/
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 @@ public class BlockManagerTestUtil {
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 class BlockManagerTestUtil {
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 class TestBlockInfo {
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 class TestBlockInfo {
// 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 class TestBlockInfo {
}
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 class TestBlockInfo {
}
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 @@ import org.junit.Test;
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.assertFalse;
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.protocol.HdfsConstants;
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.LinkedListMultimap;
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,28 +85,29 @@ public class TestBlockManager {
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,
0L, 0L,
0, 0);
BlockManagerTestUtil.getStorageReportsForDatanode(dn), 0L, 0L, 0, 0);
bm.getDatanodeManager().checkIfClusterIsNowMultiRack(dn);
}
}
@ -125,17 +132,18 @@ public class TestBlockManager {
}
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()));
}
@ -156,21 +164,22 @@ public class TestBlockManager {
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;
}
@ -199,22 +208,23 @@ public class TestBlockManager {
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)) {
@ -251,21 +261,22 @@ public class TestBlockManager {
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;
}
@ -284,11 +295,12 @@ public class TestBlockManager {
// 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);
}
@ -309,30 +321,30 @@ public class TestBlockManager {
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);
}
@ -342,7 +354,7 @@ public class TestBlockManager {
List<DatanodeDescriptor> origNodes)
throws Exception {
assertEquals(0, bm.numOfUnderReplicatedBlocks());
addBlockOnNodes((long)testIndex, origNodes);
addBlockOnNodes(testIndex, origNodes);
bm.processMisReplicatedBlocks();
assertEquals(0, bm.numOfUnderReplicatedBlocks());
}
@ -353,9 +365,11 @@ public class TestBlockManager {
* 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);
}
}
@ -364,7 +378,9 @@ public class TestBlockManager {
BlockInfo blockInfo = new BlockInfo(block, 3);
for (DatanodeDescriptor dn : nodes) {
blockInfo.addNode(dn);
for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
blockInfo.addStorage(storage);
}
}
return blockInfo;
}
@ -376,6 +392,22 @@ public class TestBlockManager {
}
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);
@ -394,7 +426,7 @@ public class TestBlockManager {
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);
@ -412,27 +444,29 @@ public class TestBlockManager {
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;
@ -455,7 +489,7 @@ public class TestBlockManager {
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"
@ -478,7 +512,7 @@ public class TestBlockManager {
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"
@ -494,7 +528,11 @@ public class TestBlockManager {
@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 =
@ -507,35 +545,40 @@ public class TestBlockManager {
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 =
@ -548,12 +591,13 @@ public class TestBlockManager {
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 class TestDatanodeDescriptor {
@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 class TestDatanodeManager {
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 class TestDatanodeManager {
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 class TestDatanodeManager {
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 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
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.BlockRecoveryCommand;
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 class TestHeartbeatHandling {
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 class TestHeartbeatHandling {
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 class TestHeartbeatHandling {
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 class TestHeartbeatHandling {
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 class TestHeartbeatHandling {
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 class TestHeartbeatHandling {
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 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
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 class TestNodeCount {
}
// 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 class TestOverReplicatedBlocks {
String corruptMachineName = corruptDataNode.getXferAddr();
for (DatanodeDescriptor datanode : hm.getDatanodes()) {
if (!corruptMachineName.equals(datanode.getXferAddr())) {
datanode.updateHeartbeat(100L, 100L, 0L, 100L, 0L, 0L, 0, 0);
datanode.getStorageInfos()[0].setUtilizationForTesting(100L, 100L, 0, 100L);
datanode.updateHeartbeat(
BlockManagerTestUtil.getStorageReportsForDatanode(datanode),
0L, 0L, 0, 0);
}
}
@ -155,7 +158,7 @@ public class TestOverReplicatedBlocks {
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 class TestOverReplicatedBlocks {
}
}
}

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.ReceivedDeletedBlockInfo.BlockStat
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 class 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 class 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 class 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 class TestPendingReplication {
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 class TestPendingReplication {
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 class TestPendingReplication {
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

@ -24,6 +24,7 @@ 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.io.IOException;
import java.util.ArrayList;
@ -35,6 +36,7 @@ import java.util.Map;
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.ContentSummary;
import org.apache.hadoop.fs.FileSystem;
@ -44,6 +46,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
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.common.HdfsServerConstants.BlockUCState;
@ -54,6 +57,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager.StatefulBlockI
import org.apache.hadoop.hdfs.server.namenode.FSClusterStats;
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;
@ -67,6 +71,10 @@ import org.junit.Test;
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;
@ -75,6 +83,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;
@ -82,17 +91,28 @@ public class TestReplicationPolicy {
@Rule
public ExpectedException exception = ExpectedException.none();
private static void updateHeartbeatWithUsage(DatanodeDescriptor dn,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
long dnCacheCapacity, long dnCacheUsed, int xceiverCount, int volFailures) {
dn.getStorageInfos()[0].setUtilizationForTesting(
capacity, dfsUsed, remaining, blockPoolUsed);
dn.updateHeartbeat(
BlockManagerTestUtil.getStorageReportsForDatanode(dn),
dnCacheCapacity, dnCacheUsed, 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");
@ -117,12 +137,19 @@ public class TestReplicationPolicy {
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, 0L, 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
@ -134,74 +161,74 @@ public class TestReplicationPolicy {
*/
@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,
0L, 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, 0L, 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);
}
/**
@ -215,8 +242,8 @@ public class TestReplicationPolicy {
@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]);
@ -228,49 +255,52 @@ public class TestReplicationPolicy {
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);
}
@ -285,41 +315,41 @@ public class TestReplicationPolicy {
@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,
0L, 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, 0L, 0L, 0, 0);
}
@ -336,35 +366,35 @@ public class TestReplicationPolicy {
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, 0L, 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, 0L, 0L, 0, 0);
}
@ -382,7 +412,7 @@ public class TestReplicationPolicy {
DatanodeDescriptor writerDesc =
DFSTestUtil.getDatanodeDescriptor("7.7.7.7", "/d2/r4");
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, writerDesc);
assertEquals(targets.length, 0);
@ -391,12 +421,12 @@ public class TestReplicationPolicy {
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]));
}
/**
@ -426,7 +456,7 @@ public class TestReplicationPolicy {
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, 0L, 0L, 0, 0);
}
@ -437,7 +467,7 @@ public class TestReplicationPolicy {
// 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();
@ -445,30 +475,42 @@ public class TestReplicationPolicy {
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, 0L, 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
@ -477,19 +519,19 @@ public class TestReplicationPolicy {
.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());
@ -514,7 +556,7 @@ public class TestReplicationPolicy {
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
@ -586,11 +628,12 @@ public class TestReplicationPolicy {
.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++) {
@ -611,10 +654,11 @@ public class TestReplicationPolicy {
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
@ -636,7 +680,7 @@ public class TestReplicationPolicy {
// Call chooseTarget
targets = chooseTarget(3, staleNodeInfo);
assertEquals(targets.length, 3);
assertFalse(cluster.isOnSameRack(targets[0], staleNodeInfo));
assertFalse(isOnSameRack(targets[0], staleNodeInfo));
} finally {
miniCluster.shutdown();
}
@ -651,26 +695,26 @@ public class TestReplicationPolicy {
*/
@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]));
}
/**
@ -682,22 +726,22 @@ public class TestReplicationPolicy {
*/
@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]));
}
/**
@ -709,31 +753,31 @@ public class TestReplicationPolicy {
*/
@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]));
}
/**
@ -1077,7 +1121,8 @@ public class TestReplicationPolicy {
// Adding this block will increase its current replication, and that will
// remove it from the queue.
bm.addStoredBlockUnderConstruction(new StatefulBlockInfo(info, info,
ReplicaState.FINALIZED), TestReplicationPolicy.dataNodes[0]);
ReplicaState.FINALIZED), TestReplicationPolicy.dataNodes[0],
"STORAGE");
// Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
// from QUEUE_VERY_UNDER_REPLICATED.
@ -1125,11 +1170,12 @@ public class TestReplicationPolicy {
info.setBlockCollection(mbc);
bm.addBlockCollection(info, mbc);
DatanodeDescriptor[] dnAry = {dataNodes[0]};
DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
dataNodes[0], new DatanodeStorage("s1"))};
final BlockInfoUnderConstruction ucBlock =
info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
dnAry);
when(mbc.setLastBlock((BlockInfo) any(), (DatanodeDescriptor[]) any()))
storageAry);
when(mbc.setLastBlock((BlockInfo) any(), (DatanodeStorageInfo[]) any()))
.thenReturn(ucBlock);
bm.convertLastBlockToUnderConstruction(mbc);

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FileSystem;
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;
@ -57,41 +58,57 @@ public class TestReplicationPolicyWithNodeGroup {
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 =
@ -129,9 +146,20 @@ public class TestReplicationPolicyWithNodeGroup {
namenode.stop();
}
private static void updateHeartbeatWithUsage(DatanodeDescriptor dn,
long capacity, long dfsUsed, long remaining, long blockPoolUsed,
long dnCacheCapacity, long dnCacheUsed, int xceiverCount,
int volFailures) {
dn.getStorageInfos()[0].setUtilizationForTesting(
capacity, dfsUsed, remaining, blockPoolUsed);
dn.updateHeartbeat(
BlockManagerTestUtil.getStorageReportsForDatanode(dn),
dnCacheCapacity, dnCacheUsed, 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, 0L, 0L, 0, 0);
}
@ -142,11 +170,12 @@ public class TestReplicationPolicyWithNodeGroup {
* 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;
@ -156,34 +185,50 @@ public class TestReplicationPolicyWithNodeGroup {
}
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);
}
/**
@ -197,49 +242,53 @@ public class TestReplicationPolicyWithNodeGroup {
*/
@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,
0L, 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, 0L, 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);
}
@ -254,36 +303,37 @@ public class TestReplicationPolicyWithNodeGroup {
*/
@Test
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);
}
@ -298,39 +348,39 @@ public class TestReplicationPolicyWithNodeGroup {
@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,
0L, 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, 0L, 0L, 0, 0);
}
@ -348,33 +398,33 @@ public class TestReplicationPolicyWithNodeGroup {
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, 0L, 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]));
}
/**
@ -387,7 +437,7 @@ public class TestReplicationPolicyWithNodeGroup {
@Test
public void testChooseTarget5() throws Exception {
setupDataNodeCapacity();
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, NODE);
assertEquals(targets.length, 0);
@ -396,12 +446,12 @@ public class TestReplicationPolicyWithNodeGroup {
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);
}
@ -415,27 +465,27 @@ public class TestReplicationPolicyWithNodeGroup {
@Test
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]));
}
/**
@ -448,22 +498,22 @@ public class TestReplicationPolicyWithNodeGroup {
@Test
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]));
}
/**
@ -476,33 +526,33 @@ public class TestReplicationPolicyWithNodeGroup {
@Test
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]));
}
/**
@ -576,16 +626,17 @@ public class TestReplicationPolicyWithNodeGroup {
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, 0L, 0L, 0, 0);
dataNodesInBoundaryCase[i].updateHeartbeat(
(HdfsConstants.MIN_BLOCKS_FOR_WRITE-1)*BLOCK_SIZE,
0L, 0L, 0L, 0, 0);
updateHeartbeatWithUsage(dataNodesInBoundaryCase[i],
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L,
2*HdfsConstants.MIN_BLOCKS_FOR_WRITE*BLOCK_SIZE, 0L, 0L, 0L, 0, 0);
}
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
targets = chooseTarget(0, dataNodesInBoundaryCase[0]);
assertEquals(targets.length, 0);
@ -594,7 +645,7 @@ public class TestReplicationPolicyWithNodeGroup {
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);
@ -611,19 +662,17 @@ public class TestReplicationPolicyWithNodeGroup {
@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, 0L, 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));
}
@ -651,12 +700,12 @@ public class TestReplicationPolicyWithNodeGroup {
}
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, 0L, 0L, 0, 0);
}
DatanodeDescriptor[] targets;
DatanodeStorageInfo[] targets;
// Test normal case -- 3 replicas
targets = chooseTarget(3, dataNodesInMoreTargetsCase[0]);
assertEquals(targets.length, 3);

View File

@ -48,7 +48,8 @@ public class TestUnderReplicatedBlocks {
// 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);
// Compute the invalidate work in NN, and trigger the heartbeat from DN
BlockManagerTestUtil.computeAllPendingWork(bm);

View File

@ -17,14 +17,37 @@
*/
package org.apache.hadoop.hdfs.server.common;
import com.google.common.base.Strings;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.io.StringReader;
import java.net.InetSocketAddress;
import java.text.MessageFormat;
import java.util.ArrayList;
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.jsp.JspWriter;
import javax.xml.parsers.DocumentBuilder;
import javax.xml.parsers.DocumentBuilderFactory;
import javax.xml.parsers.ParserConfigurationException;
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;
@ -46,20 +69,7 @@ import org.mockito.stubbing.Answer;
import org.xml.sax.InputSource;
import org.xml.sax.SAXException;
import javax.servlet.ServletContext;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.jsp.JspWriter;
import javax.xml.parsers.DocumentBuilder;
import javax.xml.parsers.DocumentBuilderFactory;
import javax.xml.parsers.ParserConfigurationException;
import java.io.IOException;
import java.io.StringReader;
import java.net.InetSocketAddress;
import java.text.MessageFormat;
import java.util.ArrayList;
import static org.junit.Assert.*;
import static org.mockito.Mockito.*;
import com.google.common.base.Strings;
public class TestJspHelper {
@ -447,14 +457,28 @@ public class TestJspHelper {
@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, 5l, 3l, 10, 2);
DatanodeDescriptor dnDesc2 = new DatanodeDescriptor(dnId2, "rack2", 2500,
200, 1848, 200, 10l, 2l, 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, 5l, 3l, 10, 2);
dnDesc2.updateHeartbeat(report2, 10l, 2l, 20, 1);
ArrayList<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
live.add(dnDesc1);
live.add(dnDesc2);
@ -615,3 +639,4 @@ public class TestJspHelper {
MessageFormat.format(EXPECTED__NOTF_PATTERN, version)));
}
}

View File

@ -22,11 +22,11 @@ import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
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.ReplicaOutputStreams;
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 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
@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 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
}
}
@Override
public String getStorageUuid() {
return storage.getStorageUuid();
}
@Override
synchronized public long getGenerationStamp() {
return theBlock.getGenerationStamp();
@ -318,13 +325,15 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
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 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
}
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 class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
}
}
@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 class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
return new BlockListAsLongs(blocks, null);
}
@Override
public synchronized Map<DatanodeStorage, BlockListAsLongs> getBlockReports(
String bpid) {
return Collections.singletonMap(new DatanodeStorage(storage.storageUuid), getBlockReport(bpid));
}
@Override // FsDatasetSpi
public List<Long> getCacheReport(String bpid) {
return new LinkedList<Long>();
@ -661,7 +686,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
}
@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());
@ -675,6 +700,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
map.remove(b.getLocalBlock());
binfo.theBlock.setGenerationStamp(newGS);
map.put(binfo.theBlock, binfo);
return binfo.getStorageUuid();
}
@Override // FsDatasetSpi
@ -931,7 +957,7 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
@Override
public String getStorageInfo() {
return "Simulated FSDataset-" + storageId;
return "Simulated FSDataset-" + datanodeUuid;
}
@Override
@ -958,7 +984,8 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
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
@ -1012,11 +1039,6 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
throw new UnsupportedOperationException();
}
@Override
public String[] getBlockPoolList() {
throw new UnsupportedOperationException();
}
@Override
public void checkAndUpdate(String bpid, long blockId, File diskFile,
File diskMetaFile, FsVolumeSpi vol) {
@ -1029,7 +1051,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
}
@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();
}
@ -1048,3 +1075,4 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
throw new UnsupportedOperationException();
}
}

View File

@ -104,7 +104,7 @@ public class TestBPOfferService {
.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.
@ -180,7 +180,7 @@ public class TestBPOfferService {
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);

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