HDFS-7364. Balancer always shows zero Bytes Already Moved. Contributed by Tsz Wo Nicholas Sze.

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java
This commit is contained in:
Jing Zhao 2014-11-06 17:48:36 -08:00
parent 52e57a95d9
commit 10fe37522c
5 changed files with 143 additions and 45 deletions

View File

@ -897,6 +897,9 @@ Release 2.6.0 - UNRELEASED
file descriptors when SASL is enabled on DataTransferProtocol.
(Chris Nauroth via wheat9)
HDFS-7364. Balancer always shows zero Bytes Already Moved.
(Tsz Wo Nicholas Sze via jing9)
BREAKDOWN OF HDFS-6134 AND HADOOP-10150 SUBTASKS AND RELATED JIRAS
HDFS-6387. HDFS CLI admin tool for creating & deleting an

View File

@ -27,7 +27,6 @@
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.Formatter;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
@ -163,7 +162,7 @@
public class Balancer {
static final Log LOG = LogFactory.getLog(Balancer.class);
private static final Path BALANCER_ID_PATH = new Path("/system/balancer.id");
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;
@ -460,7 +459,7 @@ C chooseCandidate(G g, Iterator<C> candidates, Matcher matcher) {
}
/* reset all fields in a balancer preparing for the next iteration */
private void resetData(Configuration conf) {
void resetData(Configuration conf) {
this.overUtilized.clear();
this.aboveAvgUtilized.clear();
this.belowAvgUtilized.clear();
@ -468,16 +467,47 @@ private void resetData(Configuration conf) {
this.policy.reset();
dispatcher.reset(conf);;
}
static class Result {
final ExitStatus exitStatus;
final long bytesLeftToMove;
final long bytesBeingMoved;
final long bytesAlreadyMoved;
Result(ExitStatus exitStatus, long bytesLeftToMove, long bytesBeingMoved,
long bytesAlreadyMoved) {
this.exitStatus = exitStatus;
this.bytesLeftToMove = bytesLeftToMove;
this.bytesBeingMoved = bytesBeingMoved;
this.bytesAlreadyMoved = bytesAlreadyMoved;
}
void print(int iteration, PrintStream out) {
out.printf("%-24s %10d %19s %18s %17s%n",
DateFormat.getDateTimeInstance().format(new Date()), iteration,
StringUtils.byteDesc(bytesAlreadyMoved),
StringUtils.byteDesc(bytesLeftToMove),
StringUtils.byteDesc(bytesBeingMoved));
}
}
Result newResult(ExitStatus exitStatus, long bytesLeftToMove, long bytesBeingMoved) {
return new Result(exitStatus, bytesLeftToMove, bytesBeingMoved,
dispatcher.getBytesMoved());
}
Result newResult(ExitStatus exitStatus) {
return new Result(exitStatus, -1, -1, dispatcher.getBytesMoved());
}
/** Run an iteration for all datanodes. */
private ExitStatus run(int iteration, Formatter formatter,
Configuration conf) {
Result runOneIteration() {
try {
final List<DatanodeStorageReport> reports = dispatcher.init();
final long bytesLeftToMove = init(reports);
if (bytesLeftToMove == 0) {
System.out.println("The cluster is balanced. Exiting...");
return ExitStatus.SUCCESS;
return newResult(ExitStatus.SUCCESS, bytesLeftToMove, -1);
} else {
LOG.info( "Need to move "+ StringUtils.byteDesc(bytesLeftToMove)
+ " to make the cluster balanced." );
@ -488,22 +518,14 @@ private ExitStatus run(int iteration, Formatter formatter,
* in this iteration. Maximum bytes to be moved per node is
* Min(1 Band worth of bytes, MAX_SIZE_TO_MOVE).
*/
final long bytesToMove = chooseStorageGroups();
if (bytesToMove == 0) {
final long bytesBeingMoved = chooseStorageGroups();
if (bytesBeingMoved == 0) {
System.out.println("No block can be moved. Exiting...");
return ExitStatus.NO_MOVE_BLOCK;
return newResult(ExitStatus.NO_MOVE_BLOCK, bytesLeftToMove, bytesBeingMoved);
} else {
LOG.info( "Will move " + StringUtils.byteDesc(bytesToMove) +
LOG.info( "Will move " + StringUtils.byteDesc(bytesBeingMoved) +
" in this iteration");
}
formatter.format("%-24s %10d %19s %18s %17s%n",
DateFormat.getDateTimeInstance().format(new Date()),
iteration,
StringUtils.byteDesc(dispatcher.getBytesMoved()),
StringUtils.byteDesc(bytesLeftToMove),
StringUtils.byteDesc(bytesToMove)
);
/* For each pair of <source, target>, start a thread that repeatedly
* decide a block to be moved and its proxy source,
@ -512,19 +534,19 @@ private ExitStatus run(int iteration, Formatter formatter,
* Exit no byte has been moved for 5 consecutive iterations.
*/
if (!dispatcher.dispatchAndCheckContinue()) {
return ExitStatus.NO_MOVE_PROGRESS;
return newResult(ExitStatus.NO_MOVE_PROGRESS, bytesLeftToMove, bytesBeingMoved);
}
return ExitStatus.IN_PROGRESS;
return newResult(ExitStatus.IN_PROGRESS, bytesLeftToMove, bytesBeingMoved);
} catch (IllegalArgumentException e) {
System.out.println(e + ". Exiting ...");
return ExitStatus.ILLEGAL_ARGUMENTS;
return newResult(ExitStatus.ILLEGAL_ARGUMENTS);
} catch (IOException e) {
System.out.println(e + ". Exiting ...");
return ExitStatus.IO_EXCEPTION;
return newResult(ExitStatus.IO_EXCEPTION);
} catch (InterruptedException e) {
System.out.println(e + ". Exiting ...");
return ExitStatus.INTERRUPTED;
return newResult(ExitStatus.INTERRUPTED);
} finally {
dispatcher.shutdownNow();
}
@ -546,7 +568,6 @@ static int run(Collection<URI> namenodes, final Parameters 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");
List<NameNodeConnector> connectors = Collections.emptyList();
@ -560,14 +581,16 @@ static int run(Collection<URI> namenodes, final Parameters p,
Collections.shuffle(connectors);
for(NameNodeConnector nnc : connectors) {
final Balancer b = new Balancer(nnc, p, conf);
final ExitStatus r = b.run(iteration, formatter, conf);
final Result r = b.runOneIteration();
r.print(iteration, System.out);
// clean all lists
b.resetData(conf);
if (r == ExitStatus.IN_PROGRESS) {
if (r.exitStatus == ExitStatus.IN_PROGRESS) {
done = false;
} else if (r != ExitStatus.SUCCESS) {
} else if (r.exitStatus != ExitStatus.SUCCESS) {
//must be an error statue, return.
return r.getExitCode();
return r.exitStatus.getExitCode();
}
}

View File

@ -41,9 +41,7 @@
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicLong;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
@ -78,6 +76,7 @@
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.util.Time;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
/** Dispatching block replica moves between datanodes. */
@ -121,8 +120,6 @@ public class Dispatcher {
/** The maximum number of concurrent blocks moves at a datanode */
private final int maxConcurrentMovesPerNode;
private final AtomicLong bytesMoved = new AtomicLong();
private static class GlobalBlockMap {
private final Map<Block, DBlock> map = new HashMap<Block, DBlock>();
@ -315,7 +312,7 @@ private void dispatch() {
sendRequest(out, eb, accessToken);
receiveResponse(in);
bytesMoved.addAndGet(block.getNumBytes());
nnc.getBytesMoved().addAndGet(block.getNumBytes());
LOG.info("Successfully moved " + this);
} catch (IOException e) {
LOG.warn("Failed to move " + this + ": " + e.getMessage());
@ -805,7 +802,7 @@ public NetworkTopology getCluster() {
}
long getBytesMoved() {
return bytesMoved.get();
return nnc.getBytesMoved().get();
}
long bytesToMove() {
@ -891,7 +888,7 @@ public boolean dispatchAndCheckContinue() throws InterruptedException {
* @return the total number of bytes successfully moved in this iteration.
*/
private long dispatchBlockMoves() throws InterruptedException {
final long bytesLastMoved = bytesMoved.get();
final long bytesLastMoved = getBytesMoved();
final Future<?>[] futures = new Future<?>[sources.size()];
final Iterator<Source> i = sources.iterator();
@ -917,7 +914,7 @@ public void run() {
// wait for all block moving to be done
waitForMoveCompletion(targets);
return bytesMoved.get() - bytesLastMoved;
return getBytesMoved() - bytesLastMoved;
}
/** The sleeping period before checking if block move is completed again */

View File

@ -28,6 +28,8 @@
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -108,6 +110,7 @@ public static void setWrite2IdFile(boolean write2IdFile) {
private final Path idPath;
private final OutputStream out;
private final List<Path> targetPaths;
private final AtomicLong bytesMoved = new AtomicLong();
private int notChangedIterations = 0;
@ -148,6 +151,10 @@ public String getBlockpoolID() {
return blockpoolID;
}
AtomicLong getBytesMoved() {
return bytesMoved;
}
/** @return blocks with locations. */
public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size)
throws IOException {

View File

@ -17,7 +17,12 @@
*/
package org.apache.hadoop.hdfs.server.balancer;
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC;
import static org.apache.hadoop.hdfs.StorageType.DEFAULT;
import static org.apache.hadoop.hdfs.StorageType.RAM_DISK;
import static org.junit.Assert.assertEquals;
@ -31,6 +36,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Random;
@ -44,7 +50,15 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.*;
import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.NameNodeProxies;
import org.apache.hadoop.hdfs.StorageType;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@ -53,8 +67,10 @@
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.server.balancer.Balancer.Cli;
import org.apache.hadoop.hdfs.server.balancer.Balancer.Parameters;
import org.apache.hadoop.hdfs.server.balancer.Balancer.Result;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Tool;
@ -65,8 +81,8 @@
* This class tests if a balancer schedules tasks correctly.
*/
public class TestBalancer {
private static final Log LOG = LogFactory.getLog(
"org.apache.hadoop.hdfs.TestBalancer");
private static final Log LOG = LogFactory.getLog(TestBalancer.class);
static {
((Log4JLogger)Balancer.LOG).getLogger().setLevel(Level.ALL);
}
@ -478,7 +494,7 @@ private void doTest(Configuration conf, long[] capacities,
LOG.info("racks = " + Arrays.asList(racks));
LOG.info("newCapacity= " + newCapacity);
LOG.info("newRack = " + newRack);
LOG.info("useTool = " + useTool);
LOG.info("useTool = " + useTool);
assertEquals(capacities.length, racks.length);
int numOfDatanodes = capacities.length;
cluster = new MiniDFSCluster.Builder(conf)
@ -584,7 +600,7 @@ private void runBalancer(Configuration conf,
// start rebalancing
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
final int r = Balancer.run(namenodes, p, conf);
final int r = runBalancer(namenodes, p, conf);
if (conf.getInt(DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY,
DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT) ==0) {
assertEquals(ExitStatus.NO_MOVE_PROGRESS.getExitCode(), r);
@ -593,10 +609,63 @@ private void runBalancer(Configuration conf,
assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
}
waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
LOG.info("Rebalancing with default ctor.");
LOG.info(" .");
waitForBalancer(totalUsedSpace, totalCapacity, client, cluster, p, excludedNodes);
}
private static int runBalancer(Collection<URI> namenodes, final Parameters p,
Configuration conf) throws IOException, InterruptedException {
final long sleeptime =
conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 2000 +
conf.getLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT) * 1000;
LOG.info("namenodes = " + namenodes);
LOG.info("parameters = " + p);
LOG.info("Print stack trace", new Throwable());
System.out.println("Time Stamp Iteration# Bytes Already Moved Bytes Left To Move Bytes Being Moved");
List<NameNodeConnector> connectors = Collections.emptyList();
try {
connectors = NameNodeConnector.newNameNodeConnectors(namenodes,
Balancer.class.getSimpleName(), Balancer.BALANCER_ID_PATH, conf);
boolean done = false;
for(int iteration = 0; !done; iteration++) {
done = true;
Collections.shuffle(connectors);
for(NameNodeConnector nnc : connectors) {
final Balancer b = new Balancer(nnc, p, conf);
final Result r = b.runOneIteration();
r.print(iteration, System.out);
// clean all lists
b.resetData(conf);
if (r.exitStatus == ExitStatus.IN_PROGRESS) {
done = false;
} else if (r.exitStatus != ExitStatus.SUCCESS) {
//must be an error statue, return.
return r.exitStatus.getExitCode();
} else {
if (iteration > 0) {
assertTrue(r.bytesAlreadyMoved > 0);
}
}
}
if (!done) {
Thread.sleep(sleeptime);
}
}
} finally {
for(NameNodeConnector nnc : connectors) {
IOUtils.cleanup(LOG, nnc);
}
}
return ExitStatus.SUCCESS.getExitCode();
}
private void runBalancerCli(Configuration conf,
long totalUsedSpace, long totalCapacity,
Balancer.Parameters p, boolean useFile, int expectedExcludedNodes) throws Exception {
@ -1120,7 +1189,6 @@ public void testBalancerWithRamDisk() throws Exception {
initConfWithRamDisk(conf);
final int defaultRamDiskCapacity = 10;
final int defaultDiskCapacity = 100;
final long ramDiskStorageLimit =
((long) defaultRamDiskCapacity * DEFAULT_RAM_DISK_BLOCK_SIZE) +
(DEFAULT_RAM_DISK_BLOCK_SIZE - 1);