|
@@ -82,9 +82,6 @@ import com.google.common.base.Preconditions;
|
|
|
public class Dispatcher {
|
|
|
static final Log LOG = LogFactory.getLog(Dispatcher.class);
|
|
|
|
|
|
- private static final long GB = 1L << 30; // 1GB
|
|
|
- private static final long MAX_BLOCKS_SIZE_TO_FETCH = 2 * GB;
|
|
|
-
|
|
|
private static final int MAX_NO_PENDING_MOVE_ITERATIONS = 5;
|
|
|
/**
|
|
|
* the period of time to delay the usage of a DataNode after hitting
|
|
@@ -119,6 +116,9 @@ public class Dispatcher {
|
|
|
/** The maximum number of concurrent blocks moves at a datanode */
|
|
|
private final int maxConcurrentMovesPerNode;
|
|
|
|
|
|
+ private final long getBlocksSize;
|
|
|
+ private final long getBlocksMinBlockSize;
|
|
|
+
|
|
|
private final int ioFileBufferSize;
|
|
|
|
|
|
static class Allocator {
|
|
@@ -650,8 +650,9 @@ public class Dispatcher {
|
|
|
* @return the total size of the received blocks in the number of bytes.
|
|
|
*/
|
|
|
private long getBlockList() throws IOException {
|
|
|
- final long size = Math.min(MAX_BLOCKS_SIZE_TO_FETCH, blocksToReceive);
|
|
|
+ final long size = Math.min(getBlocksSize, blocksToReceive);
|
|
|
final BlocksWithLocations newBlocks = nnc.getBlocks(getDatanodeInfo(), size);
|
|
|
+
|
|
|
if (LOG.isTraceEnabled()) {
|
|
|
LOG.trace("getBlocks(" + getDatanodeInfo() + ", "
|
|
|
+ StringUtils.TraditionalBinaryPrefix.long2String(size, "B", 2)
|
|
@@ -660,6 +661,11 @@ public class Dispatcher {
|
|
|
|
|
|
long bytesReceived = 0;
|
|
|
for (BlockWithLocations blk : newBlocks.getBlocks()) {
|
|
|
+ // Skip small blocks.
|
|
|
+ if (blk.getBlock().getNumBytes() < getBlocksMinBlockSize) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
bytesReceived += blk.getBlock().getNumBytes();
|
|
|
synchronized (globalBlocks) {
|
|
|
final DBlock block = globalBlocks.get(blk.getBlock());
|
|
@@ -838,9 +844,19 @@ public class Dispatcher {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /** Constructor called by Mover. */
|
|
|
public Dispatcher(NameNodeConnector nnc, Set<String> includedNodes,
|
|
|
Set<String> excludedNodes, long movedWinWidth, int moverThreads,
|
|
|
int dispatcherThreads, int maxConcurrentMovesPerNode, Configuration conf) {
|
|
|
+ this(nnc, includedNodes, excludedNodes, movedWinWidth,
|
|
|
+ moverThreads, dispatcherThreads, maxConcurrentMovesPerNode,
|
|
|
+ 0L, 0L, conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ Dispatcher(NameNodeConnector nnc, Set<String> includedNodes,
|
|
|
+ Set<String> excludedNodes, long movedWinWidth, int moverThreads,
|
|
|
+ int dispatcherThreads, int maxConcurrentMovesPerNode,
|
|
|
+ long getBlocksSize, long getBlocksMinBlockSize, Configuration conf) {
|
|
|
this.nnc = nnc;
|
|
|
this.excludedNodes = excludedNodes;
|
|
|
this.includedNodes = includedNodes;
|
|
@@ -853,6 +869,9 @@ public class Dispatcher {
|
|
|
this.moverThreadAllocator = new Allocator(moverThreads);
|
|
|
this.maxConcurrentMovesPerNode = maxConcurrentMovesPerNode;
|
|
|
|
|
|
+ this.getBlocksSize = getBlocksSize;
|
|
|
+ this.getBlocksMinBlockSize = getBlocksMinBlockSize;
|
|
|
+
|
|
|
this.saslClient = new SaslDataTransferClient(conf,
|
|
|
DataTransferSaslUtil.getSaslPropertiesResolver(conf),
|
|
|
TrustedChannelResolver.getInstance(conf), nnc.fallbackToSimpleAuth);
|
|
@@ -1001,9 +1020,6 @@ public class Dispatcher {
|
|
|
return getBytesMoved() - bytesLastMoved;
|
|
|
}
|
|
|
|
|
|
- /** The sleeping period before checking if block move is completed again */
|
|
|
- static private long blockMoveWaitTime = 30000L;
|
|
|
-
|
|
|
/**
|
|
|
* Wait for all block move confirmations.
|
|
|
* @return true if there is failed move execution
|
|
@@ -1025,7 +1041,7 @@ public class Dispatcher {
|
|
|
return hasFailure; // all pending queues are empty
|
|
|
}
|
|
|
try {
|
|
|
- Thread.sleep(blockMoveWaitTime);
|
|
|
+ Thread.sleep(1000);
|
|
|
} catch (InterruptedException ignored) {
|
|
|
}
|
|
|
}
|
|
@@ -1152,12 +1168,6 @@ public class Dispatcher {
|
|
|
movedBlocks.cleanup();
|
|
|
}
|
|
|
|
|
|
- /** set the sleeping period for block move completion check */
|
|
|
- @VisibleForTesting
|
|
|
- public static void setBlockMoveWaitTime(long time) {
|
|
|
- blockMoveWaitTime = time;
|
|
|
- }
|
|
|
-
|
|
|
@VisibleForTesting
|
|
|
public static void setDelayAfterErrors(long time) {
|
|
|
delayAfterErrors = time;
|