|
@@ -18,6 +18,7 @@
|
|
|
package org.apache.hadoop.hdfs.server.balancer;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed;
|
|
|
+import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength;
|
|
|
|
|
|
import java.io.BufferedInputStream;
|
|
|
import java.io.BufferedOutputStream;
|
|
@@ -52,6 +53,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
|
|
@@ -65,6 +67,7 @@ import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
@@ -124,18 +127,17 @@ public class Dispatcher {
|
|
|
private final Map<Block, DBlock> map = new HashMap<Block, DBlock>();
|
|
|
|
|
|
/**
|
|
|
- * Get the block from the map;
|
|
|
- * if the block is not found, create a new block and put it in the map.
|
|
|
+ * Put block in the map if it's not found
|
|
|
+ * @return the block which be put in the map the first time
|
|
|
*/
|
|
|
- private DBlock get(Block b) {
|
|
|
- DBlock block = map.get(b);
|
|
|
- if (block == null) {
|
|
|
- block = new DBlock(b);
|
|
|
- map.put(b, block);
|
|
|
+ private DBlock putIfAbsent(Block blk, DBlock dblk) {
|
|
|
+ if (!map.containsKey(blk)) {
|
|
|
+ map.put(blk, dblk);
|
|
|
+ return dblk;
|
|
|
}
|
|
|
- return block;
|
|
|
+ return map.get(blk);
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
/** Remove all blocks except for the moved blocks. */
|
|
|
private void removeAllButRetain(MovedBlocks<StorageGroup> movedBlocks) {
|
|
|
for (Iterator<Block> i = map.keySet().iterator(); i.hasNext();) {
|
|
@@ -176,9 +178,9 @@ public class Dispatcher {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** This class keeps track of a scheduled block move */
|
|
|
+ /** This class keeps track of a scheduled reportedBlock move */
|
|
|
public class PendingMove {
|
|
|
- private DBlock block;
|
|
|
+ private DBlock reportedBlock;
|
|
|
private Source source;
|
|
|
private DDatanode proxySource;
|
|
|
private StorageGroup target;
|
|
@@ -190,7 +192,7 @@ public class Dispatcher {
|
|
|
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
- final Block b = block != null ? block.getBlock() : null;
|
|
|
+ final Block b = reportedBlock != null ? reportedBlock.getBlock() : null;
|
|
|
String bStr = b != null ? (b + " with size=" + b.getNumBytes() + " ")
|
|
|
: " ";
|
|
|
return bStr + "from " + source.getDisplayName() + " to " + target
|
|
@@ -199,8 +201,8 @@ public class Dispatcher {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Choose a block & a proxy source for this pendingMove whose source &
|
|
|
- * target have already been chosen.
|
|
|
+ * Choose a good block/blockGroup from source & Get reportedBlock from
|
|
|
+ * the block & Choose a proxy source for the reportedBlock.
|
|
|
*
|
|
|
* @return true if a block and its proxy are chosen; false otherwise
|
|
|
*/
|
|
@@ -224,7 +226,11 @@ public class Dispatcher {
|
|
|
synchronized (block) {
|
|
|
synchronized (movedBlocks) {
|
|
|
if (isGoodBlockCandidate(source, target, targetStorageType, block)) {
|
|
|
- this.block = block;
|
|
|
+ if (block instanceof DBlockStriped) {
|
|
|
+ reportedBlock = ((DBlockStriped) block).getInternalBlock(source);
|
|
|
+ } else {
|
|
|
+ reportedBlock = block;
|
|
|
+ }
|
|
|
if (chooseProxySource()) {
|
|
|
movedBlocks.put(block);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -251,7 +257,7 @@ public class Dispatcher {
|
|
|
}
|
|
|
// if node group is supported, first try add nodes in the same node group
|
|
|
if (cluster.isNodeGroupAware()) {
|
|
|
- for (StorageGroup loc : block.getLocations()) {
|
|
|
+ for (StorageGroup loc : reportedBlock.getLocations()) {
|
|
|
if (cluster.isOnSameNodeGroup(loc.getDatanodeInfo(), targetDN)
|
|
|
&& addTo(loc)) {
|
|
|
return true;
|
|
@@ -259,13 +265,13 @@ public class Dispatcher {
|
|
|
}
|
|
|
}
|
|
|
// check if there is replica which is on the same rack with the target
|
|
|
- for (StorageGroup loc : block.getLocations()) {
|
|
|
+ for (StorageGroup loc : reportedBlock.getLocations()) {
|
|
|
if (cluster.isOnSameRack(loc.getDatanodeInfo(), targetDN) && addTo(loc)) {
|
|
|
return true;
|
|
|
}
|
|
|
}
|
|
|
// find out a non-busy replica
|
|
|
- for (StorageGroup loc : block.getLocations()) {
|
|
|
+ for (StorageGroup loc : reportedBlock.getLocations()) {
|
|
|
if (addTo(loc)) {
|
|
|
return true;
|
|
|
}
|
|
@@ -273,7 +279,7 @@ public class Dispatcher {
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- /** add to a proxy source for specific block movement */
|
|
|
+ /** add to a proxy source for specific reportedBlock movement */
|
|
|
private boolean addTo(StorageGroup g) {
|
|
|
final DDatanode dn = g.getDDatanode();
|
|
|
if (dn.addPendingBlock(this)) {
|
|
@@ -288,6 +294,7 @@ public class Dispatcher {
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Start moving " + this);
|
|
|
}
|
|
|
+ assert !(reportedBlock instanceof DBlockStriped);
|
|
|
|
|
|
Socket sock = new Socket();
|
|
|
DataOutputStream out = null;
|
|
@@ -302,7 +309,7 @@ public class Dispatcher {
|
|
|
OutputStream unbufOut = sock.getOutputStream();
|
|
|
InputStream unbufIn = sock.getInputStream();
|
|
|
ExtendedBlock eb = new ExtendedBlock(nnc.getBlockpoolID(),
|
|
|
- block.getBlock());
|
|
|
+ reportedBlock.getBlock());
|
|
|
final KeyManager km = nnc.getKeyManager();
|
|
|
Token<BlockTokenIdentifier> accessToken = km.getAccessToken(eb);
|
|
|
IOStreamPair saslStreams = saslClient.socketSend(sock, unbufOut,
|
|
@@ -316,7 +323,7 @@ public class Dispatcher {
|
|
|
|
|
|
sendRequest(out, eb, accessToken);
|
|
|
receiveResponse(in);
|
|
|
- nnc.getBytesMoved().addAndGet(block.getNumBytes());
|
|
|
+ nnc.getBytesMoved().addAndGet(reportedBlock.getNumBytes());
|
|
|
LOG.info("Successfully moved " + this);
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("Failed to move " + this + ": " + e.getMessage());
|
|
@@ -344,14 +351,14 @@ public class Dispatcher {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** Send a block replace request to the output stream */
|
|
|
+ /** Send a reportedBlock replace request to the output stream */
|
|
|
private void sendRequest(DataOutputStream out, ExtendedBlock eb,
|
|
|
Token<BlockTokenIdentifier> accessToken) throws IOException {
|
|
|
new Sender(out).replaceBlock(eb, target.storageType, accessToken,
|
|
|
source.getDatanodeInfo().getDatanodeUuid(), proxySource.datanode);
|
|
|
}
|
|
|
|
|
|
- /** Receive a block copy response from the input stream */
|
|
|
+ /** Receive a reportedBlock copy response from the input stream */
|
|
|
private void receiveResponse(DataInputStream in) throws IOException {
|
|
|
BlockOpResponseProto response =
|
|
|
BlockOpResponseProto.parseFrom(vintPrefixed(in));
|
|
@@ -359,13 +366,13 @@ public class Dispatcher {
|
|
|
// read intermediate responses
|
|
|
response = BlockOpResponseProto.parseFrom(vintPrefixed(in));
|
|
|
}
|
|
|
- String logInfo = "block move is failed";
|
|
|
+ String logInfo = "reportedBlock move is failed";
|
|
|
DataTransferProtoUtil.checkBlockOpStatus(response, logInfo);
|
|
|
}
|
|
|
|
|
|
/** reset the object */
|
|
|
private void reset() {
|
|
|
- block = null;
|
|
|
+ reportedBlock = null;
|
|
|
source = null;
|
|
|
proxySource = null;
|
|
|
target = null;
|
|
@@ -377,6 +384,44 @@ public class Dispatcher {
|
|
|
public DBlock(Block block) {
|
|
|
super(block);
|
|
|
}
|
|
|
+
|
|
|
+ public long getNumBytes(StorageGroup storage) {
|
|
|
+ return super.getNumBytes();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public static class DBlockStriped extends DBlock {
|
|
|
+
|
|
|
+ final byte[] indices;
|
|
|
+ final short dataBlockNum;
|
|
|
+
|
|
|
+ public DBlockStriped(Block block, byte[] indices, short dataBlockNum) {
|
|
|
+ super(block);
|
|
|
+ this.indices = indices;
|
|
|
+ this.dataBlockNum = dataBlockNum;
|
|
|
+ }
|
|
|
+
|
|
|
+ public DBlock getInternalBlock(StorageGroup storage) {
|
|
|
+ int idxInLocs = locations.indexOf(storage);
|
|
|
+ if (idxInLocs == -1) {
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ byte idxInGroup = indices[idxInLocs];
|
|
|
+ long blkId = getBlock().getBlockId() + idxInGroup;
|
|
|
+ long numBytes = getInternalBlockLength(getNumBytes(),
|
|
|
+ HdfsConstants.BLOCK_STRIPED_CELL_SIZE, dataBlockNum, idxInGroup);
|
|
|
+ Block blk = new Block(getBlock());
|
|
|
+ blk.setBlockId(blkId);
|
|
|
+ blk.setNumBytes(numBytes);
|
|
|
+ DBlock dblk = new DBlock(blk);
|
|
|
+ dblk.addLocation(storage);
|
|
|
+ return dblk;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public long getNumBytes(StorageGroup storage) {
|
|
|
+ return getInternalBlock(storage).getNumBytes();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/** The class represents a desired move. */
|
|
@@ -452,7 +497,7 @@ public class Dispatcher {
|
|
|
private PendingMove addPendingMove(DBlock block, final PendingMove pm) {
|
|
|
if (getDDatanode().addPendingBlock(pm)) {
|
|
|
if (pm.markMovedIfGoodBlock(block, getStorageType())) {
|
|
|
- incScheduledSize(pm.block.getNumBytes());
|
|
|
+ incScheduledSize(pm.reportedBlock.getNumBytes());
|
|
|
return pm;
|
|
|
} else {
|
|
|
getDDatanode().removePendingBlock(pm);
|
|
@@ -612,19 +657,34 @@ public class Dispatcher {
|
|
|
*/
|
|
|
private long getBlockList() throws IOException {
|
|
|
final long size = Math.min(MAX_BLOCKS_SIZE_TO_FETCH, blocksToReceive);
|
|
|
- final BlocksWithLocations newBlocks = nnc.getBlocks(getDatanodeInfo(), size);
|
|
|
+ final BlocksWithLocations newBlksLocs =
|
|
|
+ nnc.getBlocks(getDatanodeInfo(), size);
|
|
|
|
|
|
long bytesReceived = 0;
|
|
|
- for (BlockWithLocations blk : newBlocks.getBlocks()) {
|
|
|
- bytesReceived += blk.getBlock().getNumBytes();
|
|
|
+ for (BlockWithLocations blkLocs : newBlksLocs.getBlocks()) {
|
|
|
+
|
|
|
+ DBlock block;
|
|
|
+ if (blkLocs instanceof StripedBlockWithLocations) {
|
|
|
+ StripedBlockWithLocations sblkLocs =
|
|
|
+ (StripedBlockWithLocations) blkLocs;
|
|
|
+ // approximate size
|
|
|
+ bytesReceived += sblkLocs.getBlock().getNumBytes() /
|
|
|
+ sblkLocs.getDataBlockNum();
|
|
|
+ block = new DBlockStriped(sblkLocs.getBlock(), sblkLocs.getIndices(),
|
|
|
+ sblkLocs.getDataBlockNum());
|
|
|
+ } else{
|
|
|
+ bytesReceived += blkLocs.getBlock().getNumBytes();
|
|
|
+ block = new DBlock(blkLocs.getBlock());
|
|
|
+ }
|
|
|
+
|
|
|
synchronized (globalBlocks) {
|
|
|
- final DBlock block = globalBlocks.get(blk.getBlock());
|
|
|
+ block = globalBlocks.putIfAbsent(blkLocs.getBlock(), block);
|
|
|
synchronized (block) {
|
|
|
block.clearLocations();
|
|
|
|
|
|
// update locations
|
|
|
- final String[] datanodeUuids = blk.getDatanodeUuids();
|
|
|
- final StorageType[] storageTypes = blk.getStorageTypes();
|
|
|
+ final String[] datanodeUuids = blkLocs.getDatanodeUuids();
|
|
|
+ final StorageType[] storageTypes = blkLocs.getStorageTypes();
|
|
|
for (int i = 0; i < datanodeUuids.length; i++) {
|
|
|
final StorageGroup g = storageGroupMap.get(
|
|
|
datanodeUuids[i], storageTypes[i]);
|
|
@@ -661,6 +721,8 @@ public class Dispatcher {
|
|
|
* target throttling has been considered. They are chosen only when they
|
|
|
* have the capacity to support this block move. The block should be
|
|
|
* dispatched immediately after this method is returned.
|
|
|
+ * If the block is a block group. Only the internal block on this source
|
|
|
+ * will be dispatched.
|
|
|
*
|
|
|
* @return a move that's good for the source to dispatch immediately.
|
|
|
*/
|
|
@@ -672,7 +734,7 @@ public class Dispatcher {
|
|
|
if (target.addPendingBlock(pendingBlock)) {
|
|
|
// target is not busy, so do a tentative block allocation
|
|
|
if (pendingBlock.chooseBlockAndProxy()) {
|
|
|
- long blockSize = pendingBlock.block.getNumBytes();
|
|
|
+ long blockSize = pendingBlock.reportedBlock.getNumBytes(this);
|
|
|
incScheduledSize(-blockSize);
|
|
|
task.size -= blockSize;
|
|
|
if (task.size == 0) {
|
|
@@ -744,7 +806,7 @@ public class Dispatcher {
|
|
|
blocksToReceive -= getBlockList();
|
|
|
continue;
|
|
|
} catch (IOException e) {
|
|
|
- LOG.warn("Exception while getting block list", e);
|
|
|
+ LOG.warn("Exception while getting reportedBlock list", e);
|
|
|
return;
|
|
|
}
|
|
|
} else {
|
|
@@ -883,7 +945,7 @@ public class Dispatcher {
|
|
|
}
|
|
|
|
|
|
public void executePendingMove(final PendingMove p) {
|
|
|
- // move the block
|
|
|
+ // move the reportedBlock
|
|
|
moveExecutor.execute(new Runnable() {
|
|
|
@Override
|
|
|
public void run() {
|
|
@@ -928,17 +990,17 @@ public class Dispatcher {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // wait for all block moving to be done
|
|
|
+ // wait for all reportedBlock moving to be done
|
|
|
waitForMoveCompletion(targets);
|
|
|
|
|
|
return getBytesMoved() - bytesLastMoved;
|
|
|
}
|
|
|
|
|
|
- /** The sleeping period before checking if block move is completed again */
|
|
|
+ /** The sleeping period before checking if reportedBlock move is completed again */
|
|
|
static private long blockMoveWaitTime = 30000L;
|
|
|
|
|
|
/**
|
|
|
- * Wait for all block move confirmations.
|
|
|
+ * Wait for all reportedBlock move confirmations.
|
|
|
* @return true if there is failed move execution
|
|
|
*/
|
|
|
public static boolean waitForMoveCompletion(
|
|
@@ -965,10 +1027,10 @@ public class Dispatcher {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Decide if the block is a good candidate to be moved from source to target.
|
|
|
- * A block is a good candidate if
|
|
|
+ * Decide if the block/blockGroup is a good candidate to be moved from source
|
|
|
+ * to target. A block is a good candidate if
|
|
|
* 1. the block is not in the process of being moved/has not been moved;
|
|
|
- * 2. the block does not have a replica on the target;
|
|
|
+ * 2. the block does not have a replica/internalBlock on the target;
|
|
|
* 3. doing the move does not reduce the number of racks that the block has
|
|
|
*/
|
|
|
private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,
|
|
@@ -985,7 +1047,7 @@ public class Dispatcher {
|
|
|
}
|
|
|
final DatanodeInfo targetDatanode = target.getDatanodeInfo();
|
|
|
if (source.getDatanodeInfo().equals(targetDatanode)) {
|
|
|
- // the block is moved inside same DN
|
|
|
+ // the reportedBlock is moved inside same DN
|
|
|
return true;
|
|
|
}
|
|
|
|
|
@@ -1068,7 +1130,7 @@ public class Dispatcher {
|
|
|
movedBlocks.cleanup();
|
|
|
}
|
|
|
|
|
|
- /** set the sleeping period for block move completion check */
|
|
|
+ /** set the sleeping period for reportedBlock move completion check */
|
|
|
@VisibleForTesting
|
|
|
public static void setBlockMoveWaitTime(long time) {
|
|
|
blockMoveWaitTime = time;
|