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:
parent
aa7dac3359
commit
058af60c56
|
@ -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
|
||||||
|
|
|
@ -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. */
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
|
@ -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);
|
||||||
|
|
|
@ -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.
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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));
|
||||||
}
|
}
|
||||||
|
|
|
@ -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));
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
Loading…
Reference in New Issue