|
@@ -58,6 +58,7 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.conf.Configured;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
@@ -85,7 +86,6 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.net.NetworkTopology;
|
|
|
-import org.apache.hadoop.net.Node;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.util.HostsFileReader;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
@@ -195,10 +195,12 @@ import com.google.common.base.Preconditions;
|
|
|
@InterfaceAudience.Private
|
|
|
public class Balancer {
|
|
|
static final Log LOG = LogFactory.getLog(Balancer.class);
|
|
|
- final private static long GB = 1L << 30; //1GB
|
|
|
- final private static long MAX_SIZE_TO_MOVE = 10*GB;
|
|
|
- final private static long MAX_BLOCKS_SIZE_TO_FETCH = 2*GB;
|
|
|
- private static long WIN_WIDTH = 5400*1000L; // 1.5 hour
|
|
|
+
|
|
|
+ private static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
|
|
|
+
|
|
|
+ private static final long GB = 1L << 30; //1GB
|
|
|
+ private static final long MAX_SIZE_TO_MOVE = 10*GB;
|
|
|
+ private static final long MAX_BLOCKS_SIZE_TO_FETCH = 2*GB;
|
|
|
|
|
|
/** The maximum number of concurrent blocks moves for
|
|
|
* balancing purpose at a datanode
|
|
@@ -219,6 +221,8 @@ public class Balancer {
|
|
|
+ "\tIncludes only the specified datanodes.";
|
|
|
|
|
|
private final NameNodeConnector nnc;
|
|
|
+ private final KeyManager keyManager;
|
|
|
+
|
|
|
private final BalancingPolicy policy;
|
|
|
private final SaslDataTransferClient saslClient;
|
|
|
private final double threshold;
|
|
@@ -241,7 +245,8 @@ public class Balancer {
|
|
|
|
|
|
private final Map<Block, BalancerBlock> globalBlockList
|
|
|
= new HashMap<Block, BalancerBlock>();
|
|
|
- private final MovedBlocks movedBlocks = new MovedBlocks();
|
|
|
+ private final MovedBlocks<BalancerDatanode.StorageGroup> movedBlocks;
|
|
|
+
|
|
|
/** Map (datanodeUuid,storageType -> StorageGroup) */
|
|
|
private final StorageGroupMap storageGroupMap = new StorageGroupMap();
|
|
|
|
|
@@ -326,7 +331,7 @@ public class Balancer {
|
|
|
if (isGoodBlockCandidate(source, target, block)) {
|
|
|
this.block = block;
|
|
|
if ( chooseProxySource() ) {
|
|
|
- movedBlocks.add(block);
|
|
|
+ movedBlocks.put(block);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Decided to move " + this);
|
|
|
}
|
|
@@ -399,10 +404,10 @@ public class Balancer {
|
|
|
|
|
|
OutputStream unbufOut = sock.getOutputStream();
|
|
|
InputStream unbufIn = sock.getInputStream();
|
|
|
- ExtendedBlock eb = new ExtendedBlock(nnc.blockpoolID, block.getBlock());
|
|
|
- Token<BlockTokenIdentifier> accessToken = nnc.getAccessToken(eb);
|
|
|
+ ExtendedBlock eb = new ExtendedBlock(nnc.getBlockpoolID(), block.getBlock());
|
|
|
+ Token<BlockTokenIdentifier> accessToken = keyManager.getAccessToken(eb);
|
|
|
IOStreamPair saslStreams = saslClient.socketSend(sock, unbufOut,
|
|
|
- unbufIn, nnc, accessToken, target.getDatanode());
|
|
|
+ unbufIn, keyManager, accessToken, target.getDatanode());
|
|
|
unbufOut = saslStreams.out;
|
|
|
unbufIn = saslStreams.in;
|
|
|
out = new DataOutputStream(new BufferedOutputStream(unbufOut,
|
|
@@ -483,47 +488,9 @@ public class Balancer {
|
|
|
}
|
|
|
|
|
|
/* A class for keeping track of blocks in the Balancer */
|
|
|
- static private class BalancerBlock {
|
|
|
- private final Block block; // the block
|
|
|
- /** The locations of the replicas of the block. */
|
|
|
- private final List<BalancerDatanode.StorageGroup> locations
|
|
|
- = new ArrayList<BalancerDatanode.StorageGroup>(3);
|
|
|
-
|
|
|
- /* Constructor */
|
|
|
- private BalancerBlock(Block block) {
|
|
|
- this.block = block;
|
|
|
- }
|
|
|
-
|
|
|
- /* clean block locations */
|
|
|
- private synchronized void clearLocations() {
|
|
|
- locations.clear();
|
|
|
- }
|
|
|
-
|
|
|
- /* add a location */
|
|
|
- private synchronized void addLocation(BalancerDatanode.StorageGroup g) {
|
|
|
- if (!locations.contains(g)) {
|
|
|
- locations.add(g);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /** @return if the block is located on the given storage group. */
|
|
|
- private synchronized boolean isLocatedOn(BalancerDatanode.StorageGroup g) {
|
|
|
- return locations.contains(g);
|
|
|
- }
|
|
|
-
|
|
|
- /* Return its locations */
|
|
|
- private synchronized List<BalancerDatanode.StorageGroup> getLocations() {
|
|
|
- return locations;
|
|
|
- }
|
|
|
-
|
|
|
- /* Return the block */
|
|
|
- private Block getBlock() {
|
|
|
- return block;
|
|
|
- }
|
|
|
-
|
|
|
- /* Return the length of the block */
|
|
|
- private long getNumBytes() {
|
|
|
- return block.getNumBytes();
|
|
|
+ static class BalancerBlock extends MovedBlocks.Locations<BalancerDatanode.StorageGroup> {
|
|
|
+ BalancerBlock(Block block) {
|
|
|
+ super(block);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -735,7 +702,7 @@ public class Balancer {
|
|
|
*/
|
|
|
private long getBlockList() throws IOException {
|
|
|
final long size = Math.min(MAX_BLOCKS_SIZE_TO_FETCH, blocksToReceive);
|
|
|
- final BlockWithLocations[] newBlocks = nnc.namenode.getBlocks(
|
|
|
+ final BlockWithLocations[] newBlocks = nnc.getNamenode().getBlocks(
|
|
|
getDatanode(), size).getBlocks();
|
|
|
|
|
|
long bytesReceived = 0;
|
|
@@ -819,7 +786,7 @@ public class Balancer {
|
|
|
private void filterMovedBlocks() {
|
|
|
for (Iterator<BalancerBlock> blocks=getBlockIterator();
|
|
|
blocks.hasNext();) {
|
|
|
- if (movedBlocks.contains(blocks.next())) {
|
|
|
+ if (movedBlocks.contains(blocks.next().getBlock())) {
|
|
|
blocks.remove();
|
|
|
}
|
|
|
}
|
|
@@ -925,6 +892,13 @@ public class Balancer {
|
|
|
this.nodesToBeExcluded = p.nodesToBeExcluded;
|
|
|
this.nodesToBeIncluded = p.nodesToBeIncluded;
|
|
|
this.nnc = theblockpool;
|
|
|
+ this.keyManager = nnc.getKeyManager();
|
|
|
+
|
|
|
+ final long movedWinWidth = conf.getLong(
|
|
|
+ DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY,
|
|
|
+ DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_DEFAULT);
|
|
|
+ movedBlocks = new MovedBlocks<BalancerDatanode.StorageGroup>(movedWinWidth);
|
|
|
+
|
|
|
cluster = NetworkTopology.getInstance(conf);
|
|
|
|
|
|
this.moverExecutor = Executors.newFixedThreadPool(
|
|
@@ -1094,36 +1068,6 @@ public class Balancer {
|
|
|
LOG.info(items.size() + " " + name + ": " + items);
|
|
|
}
|
|
|
|
|
|
- /** A matcher interface for matching nodes. */
|
|
|
- private interface Matcher {
|
|
|
- /** Given the cluster topology, does the left node match the right node? */
|
|
|
- boolean match(NetworkTopology cluster, Node left, Node right);
|
|
|
- }
|
|
|
-
|
|
|
- /** Match datanodes in the same node group. */
|
|
|
- static final Matcher SAME_NODE_GROUP = new Matcher() {
|
|
|
- @Override
|
|
|
- public boolean match(NetworkTopology cluster, Node left, Node right) {
|
|
|
- return cluster.isOnSameNodeGroup(left, right);
|
|
|
- }
|
|
|
- };
|
|
|
-
|
|
|
- /** Match datanodes in the same rack. */
|
|
|
- static final Matcher SAME_RACK = new Matcher() {
|
|
|
- @Override
|
|
|
- public boolean match(NetworkTopology cluster, Node left, Node right) {
|
|
|
- return cluster.isOnSameRack(left, right);
|
|
|
- }
|
|
|
- };
|
|
|
-
|
|
|
- /** Match any datanode with any other datanode. */
|
|
|
- static final Matcher ANY_OTHER = new Matcher() {
|
|
|
- @Override
|
|
|
- public boolean match(NetworkTopology cluster, Node left, Node right) {
|
|
|
- return left != right;
|
|
|
- }
|
|
|
- };
|
|
|
-
|
|
|
/**
|
|
|
* Decide all <source, target> pairs and
|
|
|
* the number of bytes to move from a source to a target
|
|
@@ -1134,13 +1078,13 @@ public class Balancer {
|
|
|
private long chooseStorageGroups() {
|
|
|
// First, match nodes on the same node group if cluster is node group aware
|
|
|
if (cluster.isNodeGroupAware()) {
|
|
|
- chooseStorageGroups(SAME_NODE_GROUP);
|
|
|
+ chooseStorageGroups(Matcher.SAME_NODE_GROUP);
|
|
|
}
|
|
|
|
|
|
// Then, match nodes on the same rack
|
|
|
- chooseStorageGroups(SAME_RACK);
|
|
|
+ chooseStorageGroups(Matcher.SAME_RACK);
|
|
|
// At last, match all remaining nodes
|
|
|
- chooseStorageGroups(ANY_OTHER);
|
|
|
+ chooseStorageGroups(Matcher.ANY_OTHER);
|
|
|
|
|
|
Preconditions.checkState(storageGroupMap.size() >= sources.size() + targets.size(),
|
|
|
"Mismatched number of datanodes (" + storageGroupMap.size() + " < "
|
|
@@ -1307,56 +1251,6 @@ public class Balancer {
|
|
|
} while (shouldWait);
|
|
|
}
|
|
|
|
|
|
- /** This window makes sure to keep blocks that have been moved within 1.5 hour.
|
|
|
- * Old window has blocks that are older;
|
|
|
- * Current window has blocks that are more recent;
|
|
|
- * Cleanup method triggers the check if blocks in the old window are
|
|
|
- * more than 1.5 hour old. If yes, purge the old window and then
|
|
|
- * move blocks in current window to old window.
|
|
|
- */
|
|
|
- private static class MovedBlocks {
|
|
|
- private long lastCleanupTime = Time.now();
|
|
|
- final private static int CUR_WIN = 0;
|
|
|
- final private static int OLD_WIN = 1;
|
|
|
- final private static int NUM_WINS = 2;
|
|
|
- final private List<HashMap<Block, BalancerBlock>> movedBlocks =
|
|
|
- new ArrayList<HashMap<Block, BalancerBlock>>(NUM_WINS);
|
|
|
-
|
|
|
- /* initialize the moved blocks collection */
|
|
|
- private MovedBlocks() {
|
|
|
- movedBlocks.add(new HashMap<Block,BalancerBlock>());
|
|
|
- movedBlocks.add(new HashMap<Block,BalancerBlock>());
|
|
|
- }
|
|
|
-
|
|
|
- /* add a block thus marking a block to be moved */
|
|
|
- synchronized private void add(BalancerBlock block) {
|
|
|
- movedBlocks.get(CUR_WIN).put(block.getBlock(), block);
|
|
|
- }
|
|
|
-
|
|
|
- /* check if a block is marked as moved */
|
|
|
- synchronized private boolean contains(BalancerBlock block) {
|
|
|
- return contains(block.getBlock());
|
|
|
- }
|
|
|
-
|
|
|
- /* check if a block is marked as moved */
|
|
|
- synchronized private boolean contains(Block block) {
|
|
|
- return movedBlocks.get(CUR_WIN).containsKey(block) ||
|
|
|
- movedBlocks.get(OLD_WIN).containsKey(block);
|
|
|
- }
|
|
|
-
|
|
|
- /* remove old blocks */
|
|
|
- synchronized private void cleanup() {
|
|
|
- long curTime = Time.now();
|
|
|
- // check if old win is older than winWidth
|
|
|
- if (lastCleanupTime + WIN_WIDTH <= curTime) {
|
|
|
- // purge the old window
|
|
|
- movedBlocks.set(OLD_WIN, movedBlocks.get(CUR_WIN));
|
|
|
- movedBlocks.set(CUR_WIN, new HashMap<Block, BalancerBlock>());
|
|
|
- lastCleanupTime = curTime;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/* Decide if it is OK to move the given block 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;
|
|
@@ -1369,7 +1263,7 @@ public class Balancer {
|
|
|
return false;
|
|
|
}
|
|
|
// check if the block is moved or not
|
|
|
- if (movedBlocks.contains(block)) {
|
|
|
+ if (movedBlocks.contains(block.getBlock())) {
|
|
|
return false;
|
|
|
}
|
|
|
if (block.isLocatedOn(target)) {
|
|
@@ -1387,7 +1281,7 @@ public class Balancer {
|
|
|
} else {
|
|
|
boolean notOnSameRack = true;
|
|
|
synchronized (block) {
|
|
|
- for (BalancerDatanode.StorageGroup loc : block.locations) {
|
|
|
+ for (BalancerDatanode.StorageGroup loc : block.getLocations()) {
|
|
|
if (cluster.isOnSameRack(loc.getDatanode(), target.getDatanode())) {
|
|
|
notOnSameRack = false;
|
|
|
break;
|
|
@@ -1399,7 +1293,7 @@ public class Balancer {
|
|
|
goodBlock = true;
|
|
|
} else {
|
|
|
// good if source is on the same rack as on of the replicas
|
|
|
- for (BalancerDatanode.StorageGroup loc : block.locations) {
|
|
|
+ for (BalancerDatanode.StorageGroup loc : block.getLocations()) {
|
|
|
if (loc != source &&
|
|
|
cluster.isOnSameRack(loc.getDatanode(), source.getDatanode())) {
|
|
|
goodBlock = true;
|
|
@@ -1425,7 +1319,7 @@ public class Balancer {
|
|
|
private boolean isOnSameNodeGroupWithReplicas(BalancerDatanode.StorageGroup target,
|
|
|
BalancerBlock block, Source source) {
|
|
|
final DatanodeInfo targetDn = target.getDatanode();
|
|
|
- for (BalancerDatanode.StorageGroup loc : block.locations) {
|
|
|
+ for (BalancerDatanode.StorageGroup loc : block.getLocations()) {
|
|
|
if (loc != source &&
|
|
|
cluster.isOnSameNodeGroup(loc.getDatanode(), targetDn)) {
|
|
|
return true;
|
|
@@ -1489,7 +1383,7 @@ public class Balancer {
|
|
|
* decide the number of bytes need to be moved
|
|
|
*/
|
|
|
final long bytesLeftToMove = init(
|
|
|
- nnc.client.getDatanodeStorageReport(DatanodeReportType.LIVE));
|
|
|
+ nnc.getClient().getDatanodeStorageReport(DatanodeReportType.LIVE));
|
|
|
if (bytesLeftToMove == 0) {
|
|
|
System.out.println("The cluster is balanced. Exiting...");
|
|
|
return ReturnStatus.SUCCESS;
|
|
@@ -1558,8 +1452,8 @@ public class Balancer {
|
|
|
final long sleeptime = 2000*conf.getLong(
|
|
|
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
|
|
|
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT);
|
|
|
- LOG.info("namenodes = " + namenodes);
|
|
|
- LOG.info("p = " + p);
|
|
|
+ LOG.info("namenodes = " + namenodes);
|
|
|
+ LOG.info("parameters = " + p);
|
|
|
|
|
|
final Formatter formatter = new Formatter(System.out);
|
|
|
System.out.println("Time Stamp Iteration# Bytes Already Moved Bytes Left To Move Bytes Being Moved");
|
|
@@ -1568,7 +1462,10 @@ public class Balancer {
|
|
|
= new ArrayList<NameNodeConnector>(namenodes.size());
|
|
|
try {
|
|
|
for (URI uri : namenodes) {
|
|
|
- connectors.add(new NameNodeConnector(uri, conf));
|
|
|
+ final NameNodeConnector nnc = new NameNodeConnector(
|
|
|
+ Balancer.class.getSimpleName(), uri, BALANCER_ID_PATH, conf);
|
|
|
+ nnc.getKeyManager().startBlockKeyUpdater();
|
|
|
+ connectors.add(nnc);
|
|
|
}
|
|
|
|
|
|
boolean done = false;
|
|
@@ -1730,9 +1627,6 @@ public class Balancer {
|
|
|
public int run(String[] args) {
|
|
|
final long startTime = Time.now();
|
|
|
final Configuration conf = getConf();
|
|
|
- WIN_WIDTH = conf.getLong(
|
|
|
- DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY,
|
|
|
- DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_DEFAULT);
|
|
|
|
|
|
try {
|
|
|
checkReplicationPolicyCompatibility(conf);
|