|
@@ -39,12 +39,15 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
|
|
|
import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
|
|
|
import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertNotNull;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.Arrays;
|
|
|
|
|
|
|
|
|
/**
|
|
@@ -125,39 +128,64 @@ public class TestDataNodeErasureCodingMetrics {
|
|
|
DataNode workerDn = null;
|
|
|
DatanodeInfo[] locations = lastBlock.getLocations();
|
|
|
assertEquals(locations.length, GROUPSIZE);
|
|
|
+
|
|
|
// we have ONE extra datanode in addition to the GROUPSIZE datanodes, here
|
|
|
// is to find the extra datanode that the reconstruction task will run on,
|
|
|
// according to the current block placement logic for striped files.
|
|
|
// This can be improved later to be flexible regardless wherever the task
|
|
|
// runs.
|
|
|
- for (DataNode dn: cluster.getDataNodes()) {
|
|
|
+ for (DataNode dn : cluster.getDataNodes()) {
|
|
|
boolean appear = false;
|
|
|
- for (DatanodeInfo info: locations) {
|
|
|
+ for (DatanodeInfo info : locations) {
|
|
|
if (dn.getDatanodeUuid().equals(info.getDatanodeUuid())) {
|
|
|
appear = true;
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
- if(!appear) {
|
|
|
+ if (!appear) {
|
|
|
workerDn = dn;
|
|
|
break;
|
|
|
}
|
|
|
}
|
|
|
- byte[] indices = lastBlock.getBlockIndices();
|
|
|
- //corrupt the first block
|
|
|
- DataNode toCorruptDn = cluster.getDataNodes().get(indices[0]);
|
|
|
+ // Get a datanode from the block locations.
|
|
|
+ LOG.info("Block locations: " + Arrays.asList(locations));
|
|
|
+ LOG.info("Erasure coding worker datanode: " + workerDn);
|
|
|
+ assertNotNull("Failed to find a worker datanode", workerDn);
|
|
|
+
|
|
|
+ DataNode toCorruptDn = cluster.getDataNode(locations[0].getIpcPort());
|
|
|
+ LOG.info("Datanode to be corrupted: " + toCorruptDn);
|
|
|
+ assertNotNull("Failed to find a datanode to be corrupted", toCorruptDn);
|
|
|
toCorruptDn.shutdown();
|
|
|
setDataNodeDead(toCorruptDn.getDatanodeId());
|
|
|
DFSTestUtil.waitForDatanodeState(cluster, toCorruptDn.getDatanodeUuid(),
|
|
|
- false, 10000 );
|
|
|
- final BlockManager bm = cluster.getNamesystem().getBlockManager();
|
|
|
- BlockManagerTestUtil.getComputedDatanodeWork(bm);
|
|
|
+ false, 10000);
|
|
|
+
|
|
|
+ int workCount = getComputedDatanodeWork();
|
|
|
+ assertTrue("Wrongly computed block reconstruction work", workCount > 0);
|
|
|
cluster.triggerHeartbeats();
|
|
|
StripedFileTestUtil.waitForReconstructionFinished(file, fs, GROUPSIZE);
|
|
|
|
|
|
return workerDn;
|
|
|
}
|
|
|
|
|
|
+ private int getComputedDatanodeWork()
|
|
|
+ throws IOException, InterruptedException {
|
|
|
+ final BlockManager bm = cluster.getNamesystem().getBlockManager();
|
|
|
+ // Giving a grace period to compute datanode work.
|
|
|
+ int workCount = 0;
|
|
|
+ int retries = 20;
|
|
|
+ while (retries > 0) {
|
|
|
+ workCount = BlockManagerTestUtil.getComputedDatanodeWork(bm);
|
|
|
+ if (workCount > 0) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ retries--;
|
|
|
+ Thread.sleep(500);
|
|
|
+ }
|
|
|
+ LOG.info("Computed datanode work: " + workCount + ", retries: " + retries);
|
|
|
+ return workCount;
|
|
|
+ }
|
|
|
+
|
|
|
private void setDataNodeDead(DatanodeID dnID) throws IOException {
|
|
|
DatanodeDescriptor dnd =
|
|
|
NameNodeAdapter.getDatanode(cluster.getNamesystem(), dnID);
|