HDFS-7310. Mover can give first priority to local DN if it has target storage type available in local DN. (Vinayakumar B via umamahesh)

This commit is contained in:
Uma Maheswara Rao G 2014-11-26 23:27:25 +05:30
parent aa7dac3359
commit 058af60c56
17 changed files with 399 additions and 152 deletions

View File

@ -401,6 +401,9 @@ Release 2.7.0 - UNRELEASED
HDFS-6803 Document DFSClient#DFSInputStream expectations reading and preading HDFS-6803 Document DFSClient#DFSInputStream expectations reading and preading
in concurrent context. (stack via stevel) in concurrent context. (stack via stevel)
HDFS-7310. Mover can give first priority to local DN if it has target storage type
available in local DN. (Vinayakumar B via umamahesh)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES

View File

@ -243,6 +243,10 @@ public class Dispatcher {
*/ */
private boolean chooseProxySource() { private boolean chooseProxySource() {
final DatanodeInfo targetDN = target.getDatanodeInfo(); final DatanodeInfo targetDN = target.getDatanodeInfo();
// if source and target are same nodes then no need of proxy
if (source.getDatanodeInfo().equals(targetDN) && addTo(source)) {
return true;
}
// if node group is supported, first try add nodes in the same node group // if node group is supported, first try add nodes in the same node group
if (cluster.isNodeGroupAware()) { if (cluster.isNodeGroupAware()) {
for (StorageGroup loc : block.getLocations()) { for (StorageGroup loc : block.getLocations()) {
@ -375,19 +379,6 @@ public class Dispatcher {
public DBlock(Block block) { public DBlock(Block block) {
super(block); super(block);
} }
@Override
public synchronized boolean isLocatedOn(StorageGroup loc) {
// currently we only check if replicas are located on the same DataNodes
// since we do not have the capability to store two replicas in the same
// DataNode even though they are on two different storage types
for (StorageGroup existing : locations) {
if (existing.getDatanodeInfo().equals(loc.getDatanodeInfo())) {
return true;
}
}
return false;
}
} }
/** The class represents a desired move. */ /** The class represents a desired move. */

View File

@ -54,7 +54,6 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.fs.FileEncryptionInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException; import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
@ -63,6 +62,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.Acces
import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey; 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.block.ExportedBlockKeys;
import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason; import org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@ -2000,8 +2000,9 @@ public class BlockManager {
// place a delimiter in the list which separates blocks // place a delimiter in the list which separates blocks
// that have been reported from those that have not // that have been reported from those that have not
BlockInfo delimiter = new BlockInfo(new Block(), (short) 1); BlockInfo delimiter = new BlockInfo(new Block(), (short) 1);
boolean added = storageInfo.addBlock(delimiter); AddBlockResult result = storageInfo.addBlock(delimiter);
assert added : "Delimiting block cannot be present in the node"; assert result == AddBlockResult.ADDED
: "Delimiting block cannot be present in the node";
int headIndex = 0; //currently the delimiter is in the head of the list int headIndex = 0; //currently the delimiter is in the head of the list
int curIndex; int curIndex;
@ -2394,14 +2395,19 @@ public class BlockManager {
assert bc != null : "Block must belong to a file"; assert bc != null : "Block must belong to a file";
// add block to the datanode // add block to the datanode
boolean added = storageInfo.addBlock(storedBlock); AddBlockResult result = storageInfo.addBlock(storedBlock);
int curReplicaDelta; int curReplicaDelta;
if (added) { if (result == AddBlockResult.ADDED) {
curReplicaDelta = 1; curReplicaDelta = 1;
if (logEveryBlock) { if (logEveryBlock) {
logAddStoredBlock(storedBlock, node); logAddStoredBlock(storedBlock, node);
} }
} else if (result == AddBlockResult.REPLACED) {
curReplicaDelta = 0;
blockLog.warn("BLOCK* addStoredBlock: " + "block " + storedBlock
+ " moved to storageType " + storageInfo.getStorageType()
+ " on node " + node);
} else { } else {
// if the same block is added again and the replica was corrupt // if the same block is added again and the replica was corrupt
// previously because of a wrong gen stamp, remove it from the // previously because of a wrong gen stamp, remove it from the
@ -2423,7 +2429,7 @@ public class BlockManager {
if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED && if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
numLiveReplicas >= minReplication) { numLiveReplicas >= minReplication) {
storedBlock = completeBlock(bc, storedBlock, false); storedBlock = completeBlock(bc, storedBlock, false);
} else if (storedBlock.isComplete() && added) { } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
// check whether safe replication is reached for the block // check whether safe replication is reached for the block
// only complete blocks are counted towards that // only complete blocks are counted towards that
// Is no-op if not in safe mode. // Is no-op if not in safe mode.

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
import java.util.Iterator; import java.util.Iterator;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.apache.hadoop.util.GSet; import org.apache.hadoop.util.GSet;
import org.apache.hadoop.util.LightWeightGSet; import org.apache.hadoop.util.LightWeightGSet;
@ -223,8 +224,9 @@ class BlocksMap {
final boolean removed = storage.removeBlock(currentBlock); final boolean removed = storage.removeBlock(currentBlock);
Preconditions.checkState(removed, "currentBlock not found."); Preconditions.checkState(removed, "currentBlock not found.");
final boolean added = storage.addBlock(newBlock); final AddBlockResult result = storage.addBlock(newBlock);
Preconditions.checkState(added, "newBlock already exists."); Preconditions.checkState(result == AddBlockResult.ADDED,
"newBlock already exists.");
} }
// replace block in the map itself // replace block in the map itself
blocks.put(newBlock); blocks.put(newBlock);

View File

@ -215,10 +215,10 @@ public class DatanodeStorageInfo {
return blockPoolUsed; return blockPoolUsed;
} }
public boolean addBlock(BlockInfo b) { public AddBlockResult addBlock(BlockInfo b) {
// First check whether the block belongs to a different storage // First check whether the block belongs to a different storage
// on the same DN. // on the same DN.
boolean replaced = false; AddBlockResult result = AddBlockResult.ADDED;
DatanodeStorageInfo otherStorage = DatanodeStorageInfo otherStorage =
b.findStorageInfo(getDatanodeDescriptor()); b.findStorageInfo(getDatanodeDescriptor());
@ -226,10 +226,10 @@ public class DatanodeStorageInfo {
if (otherStorage != this) { if (otherStorage != this) {
// The block belongs to a different storage. Remove it first. // The block belongs to a different storage. Remove it first.
otherStorage.removeBlock(b); otherStorage.removeBlock(b);
replaced = true; result = AddBlockResult.REPLACED;
} else { } else {
// The block is already associated with this storage. // The block is already associated with this storage.
return false; return AddBlockResult.ALREADY_EXIST;
} }
} }
@ -237,7 +237,7 @@ public class DatanodeStorageInfo {
b.addStorage(this); b.addStorage(this);
blockList = b.listInsert(blockList, this); blockList = b.listInsert(blockList, this);
numBlocks++; numBlocks++;
return !replaced; return result;
} }
boolean removeBlock(BlockInfo b) { boolean removeBlock(BlockInfo b) {
@ -358,4 +358,8 @@ public class DatanodeStorageInfo {
} }
return null; return null;
} }
static enum AddBlockResult {
ADDED, REPLACED, ALREADY_EXIST;
}
} }

View File

@ -1007,7 +1007,6 @@ class DataXceiver extends Receiver implements Runnable {
updateCurrentThreadName("Replacing block " + block + " from " + delHint); updateCurrentThreadName("Replacing block " + block + " from " + delHint);
/* read header */ /* read header */
block.setNumBytes(dataXceiverServer.estimateBlockSize);
if (datanode.isBlockTokenEnabled) { if (datanode.isBlockTokenEnabled) {
try { try {
datanode.blockPoolTokenSecretManager.checkAccess(blockToken, null, block, datanode.blockPoolTokenSecretManager.checkAccess(blockToken, null, block,
@ -1039,6 +1038,16 @@ class DataXceiver extends Receiver implements Runnable {
DataOutputStream replyOut = new DataOutputStream(getOutputStream()); DataOutputStream replyOut = new DataOutputStream(getOutputStream());
boolean IoeDuringCopyBlockOperation = false; boolean IoeDuringCopyBlockOperation = false;
try { try {
// Move the block to different storage in the same datanode
if (proxySource.equals(datanode.getDatanodeId())) {
ReplicaInfo oldReplica = datanode.data.moveBlockAcrossStorage(block,
storageType);
if (oldReplica != null) {
LOG.info("Moved " + block + " from StorageType "
+ oldReplica.getVolume().getStorageType() + " to " + storageType);
}
} else {
block.setNumBytes(dataXceiverServer.estimateBlockSize);
// get the output stream to the proxy // get the output stream to the proxy
final String dnAddr = proxySource.getXferAddr(connectToDnViaHostname); final String dnAddr = proxySource.getXferAddr(connectToDnViaHostname);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
@ -1105,7 +1114,7 @@ class DataXceiver extends Receiver implements Runnable {
LOG.info("Moved " + block + " from " + peer.getRemoteAddressString() LOG.info("Moved " + block + " from " + peer.getRemoteAddressString()
+ ", delHint=" + delHint); + ", delHint=" + delHint);
}
} catch (IOException ioe) { } catch (IOException ioe) {
opStatus = ERROR; opStatus = ERROR;
errMsg = "opReplaceBlock " + block + " received exception " + ioe; errMsg = "opReplaceBlock " + block + " received exception " + ioe;
@ -1117,7 +1126,7 @@ class DataXceiver extends Receiver implements Runnable {
throw ioe; throw ioe;
} finally { } finally {
// receive the last byte that indicates the proxy released its thread resource // receive the last byte that indicates the proxy released its thread resource
if (opStatus == SUCCESS) { if (opStatus == SUCCESS && proxyReply != null) {
try { try {
proxyReply.readChar(); proxyReply.readChar();
} catch (IOException ignored) { } catch (IOException ignored) {

View File

@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataStorage;
import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica; import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
import org.apache.hadoop.hdfs.server.datanode.Replica; import org.apache.hadoop.hdfs.server.datanode.Replica;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface; import org.apache.hadoop.hdfs.server.datanode.ReplicaInPipelineInterface;
import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException; import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation; import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
import org.apache.hadoop.hdfs.server.datanode.UnexpectedReplicaStateException; import org.apache.hadoop.hdfs.server.datanode.UnexpectedReplicaStateException;
@ -508,4 +509,10 @@ public interface FsDatasetSpi<V extends FsVolumeSpi> extends FSDatasetMBean {
* Callback from RamDiskAsyncLazyPersistService upon async lazy persist task fail * Callback from RamDiskAsyncLazyPersistService upon async lazy persist task fail
*/ */
public void onFailLazyPersist(String bpId, long blockId); public void onFailLazyPersist(String bpId, long blockId);
/**
* Move block from one storage to another storage
*/
public ReplicaInfo moveBlockAcrossStorage(final ExtendedBlock block,
StorageType targetStorageType) throws IOException;
} }

View File

@ -157,6 +157,10 @@ class BlockPoolSlice {
return rbwDir; return rbwDir;
} }
File getTmpDir() {
return tmpDir;
}
/** Run DU on local drives. It must be synchronized from caller. */ /** Run DU on local drives. It must be synchronized from caller. */
void decDfsUsed(long value) { void decDfsUsed(long value) {
dfsUsage.decDfsUsed(value); dfsUsage.decDfsUsed(value);

View File

@ -663,13 +663,21 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
* @return the new meta and block files. * @return the new meta and block files.
* @throws IOException * @throws IOException
*/ */
static File[] copyBlockFiles(long blockId, long genStamp, static File[] copyBlockFiles(long blockId, long genStamp, File srcMeta,
File srcMeta, File srcFile, File destRoot) File srcFile, File destRoot, boolean calculateChecksum)
throws IOException { throws IOException {
final File destDir = DatanodeUtil.idToBlockDir(destRoot, blockId); final File destDir = DatanodeUtil.idToBlockDir(destRoot, blockId);
final File dstFile = new File(destDir, srcFile.getName()); final File dstFile = new File(destDir, srcFile.getName());
final File dstMeta = FsDatasetUtil.getMetaFile(dstFile, genStamp); final File dstMeta = FsDatasetUtil.getMetaFile(dstFile, genStamp);
if (calculateChecksum) {
computeChecksum(srcMeta, dstMeta, srcFile); computeChecksum(srcMeta, dstMeta, srcFile);
} else {
try {
Storage.nativeCopyFileUnbuffered(srcMeta, dstMeta, true);
} catch (IOException e) {
throw new IOException("Failed to copy " + srcMeta + " to " + dstMeta, e);
}
}
try { try {
Storage.nativeCopyFileUnbuffered(srcFile, dstFile, true); Storage.nativeCopyFileUnbuffered(srcFile, dstFile, true);
@ -677,13 +685,72 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
throw new IOException("Failed to copy " + srcFile + " to " + dstFile, e); throw new IOException("Failed to copy " + srcFile + " to " + dstFile, e);
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Copied " + srcMeta + " to " + dstMeta + if (calculateChecksum) {
" and calculated checksum"); LOG.debug("Copied " + srcMeta + " to " + dstMeta
+ " and calculated checksum");
} else {
LOG.debug("Copied " + srcFile + " to " + dstFile); LOG.debug("Copied " + srcFile + " to " + dstFile);
} }
}
return new File[] {dstMeta, dstFile}; return new File[] {dstMeta, dstFile};
} }
/**
* Move block files from one storage to another storage.
* @return Returns the Old replicaInfo
* @throws IOException
*/
@Override
public ReplicaInfo moveBlockAcrossStorage(ExtendedBlock block,
StorageType targetStorageType) throws IOException {
ReplicaInfo replicaInfo = getReplicaInfo(block);
if (replicaInfo.getState() != ReplicaState.FINALIZED) {
throw new ReplicaNotFoundException(
ReplicaNotFoundException.UNFINALIZED_REPLICA + block);
}
if (replicaInfo.getNumBytes() != block.getNumBytes()) {
throw new IOException("Corrupted replica " + replicaInfo
+ " with a length of " + replicaInfo.getNumBytes()
+ " expected length is " + block.getNumBytes());
}
if (replicaInfo.getVolume().getStorageType() == targetStorageType) {
throw new ReplicaAlreadyExistsException("Replica " + replicaInfo
+ " already exists on storage " + targetStorageType);
}
if (replicaInfo.isOnTransientStorage()) {
// Block movement from RAM_DISK will be done by LazyPersist mechanism
throw new IOException("Replica " + replicaInfo
+ " cannot be moved from storageType : "
+ replicaInfo.getVolume().getStorageType());
}
FsVolumeImpl targetVolume = volumes.getNextVolume(targetStorageType,
block.getNumBytes());
File oldBlockFile = replicaInfo.getBlockFile();
File oldMetaFile = replicaInfo.getMetaFile();
// Copy files to temp dir first
File[] blockFiles = copyBlockFiles(block.getBlockId(),
block.getGenerationStamp(), oldMetaFile, oldBlockFile,
targetVolume.getTmpDir(block.getBlockPoolId()),
replicaInfo.isOnTransientStorage());
ReplicaInfo newReplicaInfo = new ReplicaInPipeline(
replicaInfo.getBlockId(), replicaInfo.getGenerationStamp(),
targetVolume, blockFiles[0].getParentFile(), 0);
newReplicaInfo.setNumBytes(blockFiles[1].length());
// Finalize the copied files
newReplicaInfo = finalizeReplica(block.getBlockPoolId(), newReplicaInfo);
removeOldReplica(replicaInfo, newReplicaInfo, oldBlockFile, oldMetaFile,
oldBlockFile.length(), oldMetaFile.length(), block.getBlockPoolId());
// Replace the old block if any to reschedule the scanning.
datanode.getBlockScanner().addBlock(block);
return replicaInfo;
}
/** /**
* Compute and store the checksum for a block file that does not already have * Compute and store the checksum for a block file that does not already have
* its checksum computed. * its checksum computed.
@ -2442,6 +2509,35 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
} }
} }
private void removeOldReplica(ReplicaInfo replicaInfo,
ReplicaInfo newReplicaInfo, File blockFile, File metaFile,
long blockFileUsed, long metaFileUsed, final String bpid) {
// Before deleting the files from old storage we must notify the
// NN that the files are on the new storage. Else a blockReport from
// the transient storage might cause the NN to think the blocks are lost.
// Replicas must be evicted from client short-circuit caches, because the
// storage will no longer be same, and thus will require validating
// checksum. This also stops a client from holding file descriptors,
// which would prevent the OS from reclaiming the memory.
ExtendedBlock extendedBlock =
new ExtendedBlock(bpid, newReplicaInfo);
datanode.getShortCircuitRegistry().processBlockInvalidation(
ExtendedBlockId.fromExtendedBlock(extendedBlock));
datanode.notifyNamenodeReceivedBlock(
extendedBlock, null, newReplicaInfo.getStorageUuid());
// Remove the old replicas
if (blockFile.delete() || !blockFile.exists()) {
((FsVolumeImpl) replicaInfo.getVolume()).decDfsUsed(bpid, blockFileUsed);
if (metaFile.delete() || !metaFile.exists()) {
((FsVolumeImpl) replicaInfo.getVolume()).decDfsUsed(bpid, metaFileUsed);
}
}
// If deletion failed then the directory scanner will cleanup the blocks
// eventually.
}
class LazyWriter implements Runnable { class LazyWriter implements Runnable {
private volatile boolean shouldRun = true; private volatile boolean shouldRun = true;
final int checkpointerInterval; final int checkpointerInterval;
@ -2601,30 +2697,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
} }
} }
// Before deleting the files from transient storage we must notify the removeOldReplica(replicaInfo, newReplicaInfo, blockFile, metaFile,
// NN that the files are on the new storage. Else a blockReport from blockFileUsed, metaFileUsed, bpid);
// the transient storage might cause the NN to think the blocks are lost.
// Replicas must be evicted from client short-circuit caches, because the
// storage will no longer be transient, and thus will require validating
// checksum. This also stops a client from holding file descriptors,
// which would prevent the OS from reclaiming the memory.
ExtendedBlock extendedBlock =
new ExtendedBlock(bpid, newReplicaInfo);
datanode.getShortCircuitRegistry().processBlockInvalidation(
ExtendedBlockId.fromExtendedBlock(extendedBlock));
datanode.notifyNamenodeReceivedBlock(
extendedBlock, null, newReplicaInfo.getStorageUuid());
// Remove the old replicas from transient storage.
if (blockFile.delete() || !blockFile.exists()) {
((FsVolumeImpl) replicaInfo.getVolume()).decDfsUsed(bpid, blockFileUsed);
if (metaFile.delete() || !metaFile.exists()) {
((FsVolumeImpl) replicaInfo.getVolume()).decDfsUsed(bpid, metaFileUsed);
}
}
// If deletion failed then the directory scanner will cleanup the blocks
// eventually.
} }
} }

View File

@ -129,6 +129,10 @@ public class FsVolumeImpl implements FsVolumeSpi {
return getBlockPoolSlice(bpid).getLazypersistDir(); return getBlockPoolSlice(bpid).getLazypersistDir();
} }
File getTmpDir(String bpid) throws IOException {
return getBlockPoolSlice(bpid).getTmpDir();
}
void decDfsUsed(String bpid, long value) { void decDfsUsed(String bpid, long value) {
synchronized(dataset) { synchronized(dataset) {
BlockPoolSlice bp = bpSlices.get(bpid); BlockPoolSlice bp = bpSlices.get(bpid);

View File

@ -232,7 +232,7 @@ class RamDiskAsyncLazyPersistService {
try { try {
// No FsDatasetImpl lock for the file copy // No FsDatasetImpl lock for the file copy
File targetFiles[] = FsDatasetImpl.copyBlockFiles( File targetFiles[] = FsDatasetImpl.copyBlockFiles(
blockId, genStamp, metaFile, blockFile, lazyPersistDir); blockId, genStamp, metaFile, blockFile, lazyPersistDir, true);
// Lock FsDataSetImpl during onCompleteLazyPersist callback // Lock FsDataSetImpl during onCompleteLazyPersist callback
datanode.getFSDataset().onCompleteLazyPersist(bpId, blockId, datanode.getFSDataset().onCompleteLazyPersist(bpId, blockId,

View File

@ -86,8 +86,8 @@ public class Mover {
return get(sources, ml); return get(sources, ml);
} }
private StorageGroup getTarget(MLocation ml) { private StorageGroup getTarget(String uuid, StorageType storageType) {
return get(targets, ml); return targets.get(uuid, storageType);
} }
private static <G extends StorageGroup> G get(StorageGroupMap<G> map, MLocation ml) { private static <G extends StorageGroup> G get(StorageGroupMap<G> map, MLocation ml) {
@ -387,6 +387,11 @@ public class Mover {
boolean scheduleMoveReplica(DBlock db, Source source, boolean scheduleMoveReplica(DBlock db, Source source,
List<StorageType> targetTypes) { List<StorageType> targetTypes) {
// Match storage on the same node
if (chooseTargetInSameNode(db, source, targetTypes)) {
return true;
}
if (dispatcher.getCluster().isNodeGroupAware()) { if (dispatcher.getCluster().isNodeGroupAware()) {
if (chooseTarget(db, source, targetTypes, Matcher.SAME_NODE_GROUP)) { if (chooseTarget(db, source, targetTypes, Matcher.SAME_NODE_GROUP)) {
return true; return true;
@ -401,6 +406,26 @@ public class Mover {
return chooseTarget(db, source, targetTypes, Matcher.ANY_OTHER); return chooseTarget(db, source, targetTypes, Matcher.ANY_OTHER);
} }
/**
* Choose the target storage within same Datanode if possible.
*/
boolean chooseTargetInSameNode(DBlock db, Source source,
List<StorageType> targetTypes) {
for (StorageType t : targetTypes) {
StorageGroup target = storages.getTarget(source.getDatanodeInfo()
.getDatanodeUuid(), t);
if (target == null) {
continue;
}
final PendingMove pm = source.addPendingMove(db, target);
if (pm != null) {
dispatcher.executePendingMove(pm);
return true;
}
}
return false;
}
boolean chooseTarget(DBlock db, Source source, boolean chooseTarget(DBlock db, Source source,
List<StorageType> targetTypes, Matcher matcher) { List<StorageType> targetTypes, Matcher matcher) {
final NetworkTopology cluster = dispatcher.getCluster(); final NetworkTopology cluster = dispatcher.getCluster();

View File

@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
import org.junit.Assert; import org.junit.Assert;
@ -75,7 +76,8 @@ public class TestBlockInfo {
} }
// Try to move one of the blocks to a different storage. // Try to move one of the blocks to a different storage.
boolean added = storage2.addBlock(blockInfos[NUM_BLOCKS/2]); boolean added =
storage2.addBlock(blockInfos[NUM_BLOCKS / 2]) == AddBlockResult.ADDED;
Assert.assertThat(added, is(false)); Assert.assertThat(added, is(false));
Assert.assertThat(blockInfos[NUM_BLOCKS/2].getStorageInfo(0), is(storage2)); Assert.assertThat(blockInfos[NUM_BLOCKS/2].getStorageInfo(0), is(storage2));
} }

View File

@ -25,6 +25,7 @@ import java.util.ArrayList;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
import org.apache.hadoop.hdfs.server.common.GenerationStamp; import org.apache.hadoop.hdfs.server.common.GenerationStamp;
import org.junit.Test; import org.junit.Test;
@ -61,18 +62,17 @@ public class TestDatanodeDescriptor {
BlockInfo blk1 = new BlockInfo(new Block(2L), (short) 2); BlockInfo blk1 = new BlockInfo(new Block(2L), (short) 2);
DatanodeStorageInfo[] storages = dd.getStorageInfos(); DatanodeStorageInfo[] storages = dd.getStorageInfos();
assertTrue(storages.length > 0); assertTrue(storages.length > 0);
final String storageID = storages[0].getStorageID();
// add first block // add first block
assertTrue(storages[0].addBlock(blk)); assertTrue(storages[0].addBlock(blk) == AddBlockResult.ADDED);
assertEquals(1, dd.numBlocks()); assertEquals(1, dd.numBlocks());
// remove a non-existent block // remove a non-existent block
assertFalse(dd.removeBlock(blk1)); assertFalse(dd.removeBlock(blk1));
assertEquals(1, dd.numBlocks()); assertEquals(1, dd.numBlocks());
// add an existent block // add an existent block
assertFalse(storages[0].addBlock(blk)); assertFalse(storages[0].addBlock(blk) == AddBlockResult.ADDED);
assertEquals(1, dd.numBlocks()); assertEquals(1, dd.numBlocks());
// add second block // add second block
assertTrue(storages[0].addBlock(blk1)); assertTrue(storages[0].addBlock(blk1) == AddBlockResult.ADDED);
assertEquals(2, dd.numBlocks()); assertEquals(2, dd.numBlocks());
// remove first block // remove first block
assertTrue(dd.removeBlock(blk)); assertTrue(dd.removeBlock(blk));

View File

@ -1260,5 +1260,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
public void onFailLazyPersist(String bpId, long blockId) { public void onFailLazyPersist(String bpId, long blockId) {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }
@Override
public ReplicaInfo moveBlockAcrossStorage(ExtendedBlock block,
StorageType targetStorageType) throws IOException {
// TODO Auto-generated method stub
return null;
}
} }

View File

@ -17,15 +17,14 @@
*/ */
package org.apache.hadoop.hdfs.server.datanode; package org.apache.hadoop.hdfs.server.datanode;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.*;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.DataInputStream; import java.io.DataInputStream;
import java.io.DataOutputStream; import java.io.DataOutputStream;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.Socket; import java.net.Socket;
import java.net.SocketException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
@ -40,6 +39,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.StorageType; import org.apache.hadoop.hdfs.StorageType;
@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
@ -201,6 +202,50 @@ public class TestBlockReplacement {
} }
} }
@Test
public void testBlockMoveAcrossStorageInSameNode() throws Exception {
final Configuration conf = new HdfsConfiguration();
// create only one datanode in the cluster to verify movement within
// datanode.
final MiniDFSCluster cluster =
new MiniDFSCluster.Builder(conf).numDataNodes(1).storageTypes(
new StorageType[] { StorageType.DISK, StorageType.ARCHIVE })
.build();
try {
cluster.waitActive();
final DistributedFileSystem dfs = cluster.getFileSystem();
final Path file = new Path("/testBlockMoveAcrossStorageInSameNode/file");
DFSTestUtil.createFile(dfs, file, 1024, (short) 1, 1024);
LocatedBlocks locatedBlocks = dfs.getClient().getLocatedBlocks(file.toString(), 0);
// get the current
LocatedBlock locatedBlock = locatedBlocks.get(0);
ExtendedBlock block = locatedBlock.getBlock();
DatanodeInfo[] locations = locatedBlock.getLocations();
assertEquals(1, locations.length);
StorageType[] storageTypes = locatedBlock.getStorageTypes();
// current block should be written to DISK
assertTrue(storageTypes[0] == StorageType.DISK);
DatanodeInfo source = locations[0];
// move block to ARCHIVE by using same DataNodeInfo for source, proxy and
// destination so that movement happens within datanode
assertTrue(replaceBlock(block, source, source, source,
StorageType.ARCHIVE));
// wait till namenode notified
Thread.sleep(3000);
locatedBlocks = dfs.getClient().getLocatedBlocks(file.toString(), 0);
// get the current
locatedBlock = locatedBlocks.get(0);
assertEquals("Storage should be only one", 1,
locatedBlock.getLocations().length);
assertTrue("Block should be moved to ARCHIVE", locatedBlock
.getStorageTypes()[0] == StorageType.ARCHIVE);
} finally {
cluster.shutdown();
}
}
/* check if file's blocks have expected number of replicas, /* check if file's blocks have expected number of replicas,
* and exist at all of includeNodes * and exist at all of includeNodes
*/ */
@ -259,24 +304,42 @@ public class TestBlockReplacement {
*/ */
private boolean replaceBlock( ExtendedBlock block, DatanodeInfo source, private boolean replaceBlock( ExtendedBlock block, DatanodeInfo source,
DatanodeInfo sourceProxy, DatanodeInfo destination) throws IOException { DatanodeInfo sourceProxy, DatanodeInfo destination) throws IOException {
return replaceBlock(block, source, sourceProxy, destination,
StorageType.DEFAULT);
}
/*
* Replace block
*/
private boolean replaceBlock(
ExtendedBlock block,
DatanodeInfo source,
DatanodeInfo sourceProxy,
DatanodeInfo destination,
StorageType targetStorageType) throws IOException, SocketException {
Socket sock = new Socket(); Socket sock = new Socket();
sock.connect(NetUtils.createSocketAddr( try {
destination.getXferAddr()), HdfsServerConstants.READ_TIMEOUT); sock.connect(NetUtils.createSocketAddr(destination.getXferAddr()),
HdfsServerConstants.READ_TIMEOUT);
sock.setKeepAlive(true); sock.setKeepAlive(true);
// sendRequest // sendRequest
DataOutputStream out = new DataOutputStream(sock.getOutputStream()); DataOutputStream out = new DataOutputStream(sock.getOutputStream());
new Sender(out).replaceBlock(block, StorageType.DEFAULT, new Sender(out).replaceBlock(block, targetStorageType,
BlockTokenSecretManager.DUMMY_TOKEN, BlockTokenSecretManager.DUMMY_TOKEN, source.getDatanodeUuid(),
source.getDatanodeUuid(), sourceProxy); sourceProxy);
out.flush(); out.flush();
// receiveResponse // receiveResponse
DataInputStream reply = new DataInputStream(sock.getInputStream()); DataInputStream reply = new DataInputStream(sock.getInputStream());
BlockOpResponseProto proto = BlockOpResponseProto.parseDelimitedFrom(reply); BlockOpResponseProto proto =
BlockOpResponseProto.parseDelimitedFrom(reply);
while (proto.getStatus() == Status.IN_PROGRESS) { while (proto.getStatus() == Status.IN_PROGRESS) {
proto = BlockOpResponseProto.parseDelimitedFrom(reply); proto = BlockOpResponseProto.parseDelimitedFrom(reply);
} }
return proto.getStatus() == Status.SUCCESS; return proto.getStatus() == Status.SUCCESS;
} finally {
sock.close();
}
} }
/** /**

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
import org.apache.hadoop.hdfs.server.mover.Mover.MLocation; import org.apache.hadoop.hdfs.server.mover.Mover.MLocation;
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil; import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.GenericOptionsParser; import org.apache.hadoop.util.ToolRunner;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -79,6 +79,52 @@ public class TestMover {
} }
} }
@Test
public void testScheduleBlockWithinSameNode() throws Exception {
final Configuration conf = new HdfsConfiguration();
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(3)
.storageTypes(
new StorageType[] { StorageType.DISK, StorageType.ARCHIVE })
.build();
try {
cluster.waitActive();
final DistributedFileSystem dfs = cluster.getFileSystem();
final String file = "/testScheduleWithinSameNode/file";
Path dir = new Path("/testScheduleWithinSameNode");
dfs.mkdirs(dir);
// write to DISK
dfs.setStoragePolicy(dir, "HOT");
{
final FSDataOutputStream out = dfs.create(new Path(file));
out.writeChars("testScheduleWithinSameNode");
out.close();
}
//verify before movement
LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
StorageType[] storageTypes = lb.getStorageTypes();
for (StorageType storageType : storageTypes) {
Assert.assertTrue(StorageType.DISK == storageType);
}
// move to ARCHIVE
dfs.setStoragePolicy(dir, "COLD");
int rc = ToolRunner.run(conf, new Mover.Cli(),
new String[] { "-p", dir.toString() });
Assert.assertEquals("Movement to ARCHIVE should be successfull", 0, rc);
// Wait till namenode notified
Thread.sleep(3000);
lb = dfs.getClient().getLocatedBlocks(file, 0).get(0);
storageTypes = lb.getStorageTypes();
for (StorageType storageType : storageTypes) {
Assert.assertTrue(StorageType.ARCHIVE == storageType);
}
} finally {
cluster.shutdown();
}
}
private void checkMovePaths(List<Path> actual, Path... expected) { private void checkMovePaths(List<Path> actual, Path... expected) {
Assert.assertEquals(expected.length, actual.size()); Assert.assertEquals(expected.length, actual.size());
for (Path p : expected) { for (Path p : expected) {