|
@@ -17,7 +17,6 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
-import com.google.common.base.Supplier;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -28,7 +27,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
import org.apache.hadoop.hdfs.StripedFileTestUtil;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
|
|
@@ -38,21 +36,16 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
|
|
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
|
|
-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.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
-
|
|
|
import java.io.IOException;
|
|
|
-import java.util.Arrays;
|
|
|
|
|
|
/**
|
|
|
* This file tests the erasure coding metrics in DataNode.
|
|
@@ -65,8 +58,9 @@ public class TestDataNodeErasureCodingMetrics {
|
|
|
private final int dataBlocks = ecPolicy.getNumDataUnits();
|
|
|
private final int parityBlocks = ecPolicy.getNumParityUnits();
|
|
|
private final int cellSize = ecPolicy.getCellSize();
|
|
|
- private final int blockSize = cellSize;
|
|
|
+ private final int blockSize = cellSize * 2;
|
|
|
private final int groupSize = dataBlocks + parityBlocks;
|
|
|
+ private final int blockGroupSize = blockSize * dataBlocks;
|
|
|
private final int numDNs = groupSize + 1;
|
|
|
|
|
|
private MiniDFSCluster cluster;
|
|
@@ -76,7 +70,6 @@ public class TestDataNodeErasureCodingMetrics {
|
|
|
@Before
|
|
|
public void setup() throws IOException {
|
|
|
conf = new Configuration();
|
|
|
-
|
|
|
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
conf.setInt(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, 1);
|
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
|
|
@@ -93,82 +86,86 @@ public class TestDataNodeErasureCodingMetrics {
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 120000)
|
|
|
- public void testEcTasks() throws Exception {
|
|
|
- DataNode workerDn = doTest("/testEcTasks");
|
|
|
- MetricsRecordBuilder rb = getMetrics(workerDn.getMetrics().name());
|
|
|
-
|
|
|
- // Ensure that reconstruction task is finished
|
|
|
- GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
- @Override
|
|
|
- public Boolean get() {
|
|
|
- long taskMetricValue = getLongCounter("EcReconstructionTasks", rb);
|
|
|
- return (taskMetricValue > 0);
|
|
|
- }
|
|
|
- }, 500, 10000);
|
|
|
+ public void testFullBlock() throws Exception {
|
|
|
+ doTest("/testEcMetrics", blockGroupSize, 0);
|
|
|
+
|
|
|
+ Assert.assertEquals("EcReconstructionTasks should be ",
|
|
|
+ 1, getLongMetric("EcReconstructionTasks"));
|
|
|
+ Assert.assertEquals("EcFailedReconstructionTasks should be ",
|
|
|
+ 0, getLongMetric("EcFailedReconstructionTasks"));
|
|
|
+ Assert.assertTrue(getLongMetric("EcDecodingTimeNanos") > 0);
|
|
|
+ Assert.assertEquals("EcReconstructionBytesRead should be ",
|
|
|
+ blockGroupSize, getLongMetric("EcReconstructionBytesRead"));
|
|
|
+ Assert.assertEquals("EcReconstructionBytesWritten should be ",
|
|
|
+ blockSize, getLongMetric("EcReconstructionBytesWritten"));
|
|
|
+ }
|
|
|
|
|
|
- assertCounter("EcReconstructionTasks", (long) 1, rb);
|
|
|
- assertCounter("EcFailedReconstructionTasks", (long) 0, rb);
|
|
|
+ // A partial block, reconstruct the partial block
|
|
|
+ @Test(timeout = 120000)
|
|
|
+ public void testReconstructionBytesPartialGroup1() throws Exception {
|
|
|
+ final int fileLen = blockSize / 10;
|
|
|
+ doTest("/testEcBytes", fileLen, 0);
|
|
|
+
|
|
|
+ Assert.assertEquals("EcReconstructionBytesRead should be ",
|
|
|
+ fileLen, getLongMetric("EcReconstructionBytesRead"));
|
|
|
+ Assert.assertEquals("EcReconstructionBytesWritten should be ",
|
|
|
+ fileLen, getLongMetric("EcReconstructionBytesWritten"));
|
|
|
}
|
|
|
|
|
|
+ // 1 full block + 5 partial block, reconstruct the full block
|
|
|
@Test(timeout = 120000)
|
|
|
- public void testEcCodingTime() throws Exception {
|
|
|
- DataNode workerDn = doTest("/testEcCodingTime");
|
|
|
- MetricsRecordBuilder rb = getMetrics(workerDn.getMetrics().name());
|
|
|
-
|
|
|
- // Ensure that reconstruction task is finished
|
|
|
- GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
|
|
- @Override
|
|
|
- public Boolean get() {
|
|
|
- long taskMetricValue = getLongCounter("EcReconstructionTasks", rb);
|
|
|
- return (taskMetricValue > 0);
|
|
|
- }
|
|
|
- }, 500, 10000);
|
|
|
+ public void testReconstructionBytesPartialGroup2() throws Exception {
|
|
|
+ final int fileLen = cellSize * dataBlocks + cellSize + cellSize / 10;
|
|
|
+ doTest("/testEcBytes", fileLen, 0);
|
|
|
+
|
|
|
+ Assert.assertEquals("ecReconstructionBytesRead should be ",
|
|
|
+ cellSize * dataBlocks + cellSize + cellSize / 10,
|
|
|
+ getLongMetric("EcReconstructionBytesRead"));
|
|
|
+ Assert.assertEquals("ecReconstructionBytesWritten should be ",
|
|
|
+ blockSize, getLongMetric("EcReconstructionBytesWritten"));
|
|
|
+ }
|
|
|
|
|
|
- long decodeTime = getLongCounter("ecDecodingTimeNanos", rb);
|
|
|
- Assert.assertTrue(decodeTime > 0);
|
|
|
+ // 1 full block + 5 partial block, reconstruct the partial block
|
|
|
+ @Test(timeout = 120000)
|
|
|
+ public void testReconstructionBytesPartialGroup3() throws Exception {
|
|
|
+ final int fileLen = cellSize * dataBlocks + cellSize + cellSize / 10;
|
|
|
+ doTest("/testEcBytes", fileLen, 1);
|
|
|
+
|
|
|
+ Assert.assertEquals("ecReconstructionBytesRead should be ",
|
|
|
+ cellSize * dataBlocks + (cellSize / 10) * 2 ,
|
|
|
+ getLongMetric("EcReconstructionBytesRead"));
|
|
|
+ Assert.assertEquals("ecReconstructionBytesWritten should be ",
|
|
|
+ cellSize + cellSize / 10,
|
|
|
+ getLongMetric("EcReconstructionBytesWritten"));
|
|
|
}
|
|
|
|
|
|
- private DataNode doTest(String fileName) throws Exception {
|
|
|
+ private long getLongMetric(String metricName) {
|
|
|
+ long metricValue = 0;
|
|
|
+ // Add all reconstruction metric value from all data nodes
|
|
|
+ for (DataNode dn : cluster.getDataNodes()) {
|
|
|
+ MetricsRecordBuilder rb = getMetrics(dn.getMetrics().name());
|
|
|
+ metricValue += getLongCounter(metricName, rb);
|
|
|
+ }
|
|
|
+ return metricValue;
|
|
|
+ }
|
|
|
|
|
|
+ private void doTest(String fileName, int fileLen,
|
|
|
+ int deadNodeIndex) throws Exception {
|
|
|
+ assertTrue(fileLen > 0);
|
|
|
+ assertTrue(deadNodeIndex >= 0 && deadNodeIndex < numDNs);
|
|
|
Path file = new Path(fileName);
|
|
|
- long fileLen = dataBlocks * blockSize;
|
|
|
- final byte[] data = StripedFileTestUtil.generateBytes((int) fileLen);
|
|
|
+ final byte[] data = StripedFileTestUtil.generateBytes(fileLen);
|
|
|
DFSTestUtil.writeFile(fs, file, data);
|
|
|
StripedFileTestUtil.waitBlockGroupsReported(fs, fileName);
|
|
|
|
|
|
- LocatedBlocks locatedBlocks =
|
|
|
+ final LocatedBlocks locatedBlocks =
|
|
|
StripedFileTestUtil.getLocatedBlocks(file, fs);
|
|
|
- //only one block group
|
|
|
- LocatedStripedBlock lastBlock =
|
|
|
+ final LocatedStripedBlock lastBlock =
|
|
|
(LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
|
|
|
- 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()) {
|
|
|
- boolean appear = false;
|
|
|
- for (DatanodeInfo info : locations) {
|
|
|
- if (dn.getDatanodeUuid().equals(info.getDatanodeUuid())) {
|
|
|
- appear = true;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- if (!appear) {
|
|
|
- workerDn = dn;
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- // 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);
|
|
|
+ assertTrue(lastBlock.getLocations().length > deadNodeIndex);
|
|
|
|
|
|
- DataNode toCorruptDn = cluster.getDataNode(locations[0].getIpcPort());
|
|
|
+ final DataNode toCorruptDn = cluster.getDataNode(
|
|
|
+ lastBlock.getLocations()[deadNodeIndex].getIpcPort());
|
|
|
LOG.info("Datanode to be corrupted: " + toCorruptDn);
|
|
|
assertNotNull("Failed to find a datanode to be corrupted", toCorruptDn);
|
|
|
toCorruptDn.shutdown();
|
|
@@ -176,12 +173,15 @@ public class TestDataNodeErasureCodingMetrics {
|
|
|
DFSTestUtil.waitForDatanodeState(cluster, toCorruptDn.getDatanodeUuid(),
|
|
|
false, 10000);
|
|
|
|
|
|
- int workCount = getComputedDatanodeWork();
|
|
|
+ final int workCount = getComputedDatanodeWork();
|
|
|
assertTrue("Wrongly computed block reconstruction work", workCount > 0);
|
|
|
cluster.triggerHeartbeats();
|
|
|
- StripedFileTestUtil.waitForReconstructionFinished(file, fs, groupSize);
|
|
|
-
|
|
|
- return workerDn;
|
|
|
+ int totalBlocks = (fileLen / blockGroupSize) * groupSize;
|
|
|
+ final int remainder = fileLen % blockGroupSize;
|
|
|
+ totalBlocks += (remainder == 0) ? 0 :
|
|
|
+ (remainder % blockSize == 0) ? remainder / blockSize + parityBlocks :
|
|
|
+ remainder / blockSize + 1 + parityBlocks;
|
|
|
+ StripedFileTestUtil.waitForAllReconstructionFinished(file, fs, totalBlocks);
|
|
|
}
|
|
|
|
|
|
private int getComputedDatanodeWork()
|
|
@@ -209,5 +209,4 @@ public class TestDataNodeErasureCodingMetrics {
|
|
|
BlockManagerTestUtil.checkHeartbeat(
|
|
|
cluster.getNamesystem().getBlockManager());
|
|
|
}
|
|
|
-
|
|
|
}
|