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:
commit
d887c6a286
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
|||
+ "}";
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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()),
|
||||
|
|
|
@ -82,6 +82,6 @@ public class InterDatanodeProtocolServerSideTranslatorPB implements
|
|||
throw new ServiceException(e);
|
||||
}
|
||||
return UpdateReplicaUnderRecoveryResponseProto.newBuilder()
|
||||
.setStorageID(storageID).build();
|
||||
.setStorageUuid(storageID).build();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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. */
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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++;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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++;
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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? */
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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() + " "
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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,
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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");
|
||||
|
||||
|
|
|
@ -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 : "");
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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>();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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?
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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]);
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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)));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue