|
@@ -17,7 +17,9 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
+import static org.hamcrest.core.Is.is;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
+import static org.junit.Assert.assertThat;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
import java.io.File;
|
|
@@ -25,6 +27,7 @@ import java.io.FilenameFilter;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.Random;
|
|
|
import java.util.concurrent.CountDownLatch;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
@@ -89,7 +92,7 @@ public class TestBlockReport {
|
|
|
private MiniDFSCluster cluster;
|
|
|
private DistributedFileSystem fs;
|
|
|
|
|
|
- Random rand = new Random(RAND_LIMIT);
|
|
|
+ private static Random rand = new Random(RAND_LIMIT);
|
|
|
|
|
|
private static Configuration conf;
|
|
|
|
|
@@ -113,6 +116,57 @@ public class TestBlockReport {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
|
|
|
+ private static StorageBlockReport[] getBlockReports(DataNode dn, String bpid) {
|
|
|
+ Map<String, BlockListAsLongs> perVolumeBlockLists =
|
|
|
+ dn.getFSDataset().getBlockReports(bpid);
|
|
|
+
|
|
|
+ // Send block report
|
|
|
+ StorageBlockReport[] reports =
|
|
|
+ new StorageBlockReport[perVolumeBlockLists.size()];
|
|
|
+
|
|
|
+ int i = 0;
|
|
|
+ for(Map.Entry<String, BlockListAsLongs> kvPair : perVolumeBlockLists.entrySet()) {
|
|
|
+ String storageID = kvPair.getKey();
|
|
|
+ long[] blockList = kvPair.getValue().getBlockListAsLongs();
|
|
|
+
|
|
|
+ // Dummy DatanodeStorage object just for sending the block report.
|
|
|
+ DatanodeStorage dnStorage = new DatanodeStorage(storageID);
|
|
|
+ reports[i++] = new StorageBlockReport(dnStorage, blockList);
|
|
|
+ }
|
|
|
+
|
|
|
+ return reports;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Get block reports but modify the GS of one of the blocks.
|
|
|
+ private static StorageBlockReport[] getBlockReportsCorruptSingleBlockGS(
|
|
|
+ DataNode dn, String bpid) {
|
|
|
+ Map<String, BlockListAsLongs> perVolumeBlockLists =
|
|
|
+ dn.getFSDataset().getBlockReports(bpid);
|
|
|
+
|
|
|
+ // Send block report
|
|
|
+ StorageBlockReport[] reports =
|
|
|
+ new StorageBlockReport[perVolumeBlockLists.size()];
|
|
|
+
|
|
|
+ boolean corruptedBlock = false;
|
|
|
+
|
|
|
+ int i = 0;
|
|
|
+ for(Map.Entry<String, BlockListAsLongs> kvPair : perVolumeBlockLists.entrySet()) {
|
|
|
+ String storageID = kvPair.getKey();
|
|
|
+ long[] blockList = kvPair.getValue().getBlockListAsLongs();
|
|
|
+
|
|
|
+ if (!corruptedBlock) {
|
|
|
+ blockList[4] = rand.nextInt(); // Bad GS.
|
|
|
+ corruptedBlock = true;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Dummy DatanodeStorage object just for sending the block report.
|
|
|
+ DatanodeStorage dnStorage = new DatanodeStorage(storageID);
|
|
|
+ reports[i++] = new StorageBlockReport(dnStorage, blockList);
|
|
|
+ }
|
|
|
+
|
|
|
+ return reports;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Test write a file, verifies and closes it. Then the length of the blocks
|
|
|
* are messed up and BlockReport is forced.
|
|
@@ -153,10 +207,8 @@ public class TestBlockReport {
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = { new StorageBlockReport(
|
|
|
- new DatanodeStorage(dnR.getDatanodeUuid()),
|
|
|
- new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
|
|
|
- cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn, poolId);
|
|
|
+ cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
|
|
|
List<LocatedBlock> blocksAfterReport =
|
|
|
DFSTestUtil.getAllBlocks(fs.open(filePath));
|
|
@@ -211,7 +263,6 @@ public class TestBlockReport {
|
|
|
for (Integer aRemovedIndex : removedIndex) {
|
|
|
blocks2Remove.add(lBlocks.get(aRemovedIndex).getBlock());
|
|
|
}
|
|
|
- ArrayList<Block> blocks = locatedToBlocks(lBlocks, removedIndex);
|
|
|
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Number of blocks allocated " + lBlocks.size());
|
|
@@ -225,8 +276,11 @@ public class TestBlockReport {
|
|
|
for (File f : findAllFiles(dataDir,
|
|
|
new MyFileFilter(b.getBlockName(), true))) {
|
|
|
DataNodeTestUtils.getFSDataset(dn0).unfinalizeBlock(b);
|
|
|
- if (!f.delete())
|
|
|
+ if (!f.delete()) {
|
|
|
LOG.warn("Couldn't delete " + b.getBlockName());
|
|
|
+ } else {
|
|
|
+ LOG.debug("Deleted file " + f.toString());
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -235,10 +289,8 @@ public class TestBlockReport {
|
|
|
// all blocks belong to the same file, hence same BP
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn0.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = { new StorageBlockReport(
|
|
|
- new DatanodeStorage(dnR.getDatanodeUuid()),
|
|
|
- new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
|
|
|
- cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn0, poolId);
|
|
|
+ cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
|
|
|
BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
|
|
|
.getBlockManager());
|
|
@@ -253,9 +305,8 @@ public class TestBlockReport {
|
|
|
|
|
|
|
|
|
/**
|
|
|
- * Test writes a file and closes it. Then test finds a block
|
|
|
- * and changes its GS to be < of original one.
|
|
|
- * New empty block is added to the list of blocks.
|
|
|
+ * Test writes a file and closes it.
|
|
|
+ * Block reported is generated with a bad GS for a single block.
|
|
|
* Block report is forced and the check for # of corrupted blocks is performed.
|
|
|
*
|
|
|
* @throws IOException in case of an error
|
|
@@ -264,41 +315,65 @@ public class TestBlockReport {
|
|
|
public void blockReport_03() throws IOException {
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
|
-
|
|
|
- ArrayList<Block> blocks =
|
|
|
- prepareForRide(filePath, METHOD_NAME, FILE_SIZE);
|
|
|
-
|
|
|
- // The block with modified GS won't be found. Has to be deleted
|
|
|
- blocks.get(0).setGenerationStamp(rand.nextLong());
|
|
|
- // This new block is unknown to NN and will be mark for deletion.
|
|
|
- blocks.add(new Block());
|
|
|
+ DFSTestUtil.createFile(fs, filePath,
|
|
|
+ FILE_SIZE, REPL_FACTOR, rand.nextLong());
|
|
|
|
|
|
// all blocks belong to the same file, hence same BP
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = { new StorageBlockReport(
|
|
|
- new DatanodeStorage(dnR.getDatanodeUuid()),
|
|
|
- new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
|
|
|
+ StorageBlockReport[] reports = getBlockReportsCorruptSingleBlockGS(dn, poolId);
|
|
|
DatanodeCommand dnCmd =
|
|
|
- cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
+ cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
if(LOG.isDebugEnabled()) {
|
|
|
LOG.debug("Got the command: " + dnCmd);
|
|
|
}
|
|
|
printStats();
|
|
|
|
|
|
- assertEquals("Wrong number of CorruptedReplica+PendingDeletion " +
|
|
|
- "blocks is found", 2,
|
|
|
- cluster.getNamesystem().getCorruptReplicaBlocks() +
|
|
|
- cluster.getNamesystem().getPendingDeletionBlocks());
|
|
|
+ assertThat("Wrong number of corrupt blocks",
|
|
|
+ cluster.getNamesystem().getCorruptReplicaBlocks(), is(1L));
|
|
|
+ assertThat("Wrong number of PendingDeletion blocks",
|
|
|
+ cluster.getNamesystem().getPendingDeletionBlocks(), is(0L));
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * This test isn't a representative case for BlockReport
|
|
|
- * The empty method is going to be left here to keep the naming
|
|
|
- * of the test plan in synch with the actual implementation
|
|
|
+ * Test writes a file and closes it.
|
|
|
+ * Block reported is generated with an extra block.
|
|
|
+ * Block report is forced and the check for # of pendingdeletion
|
|
|
+ * blocks is performed.
|
|
|
+ *
|
|
|
+ * @throws IOException in case of an error
|
|
|
*/
|
|
|
- public void blockReport_04() {
|
|
|
+ @Test
|
|
|
+ public void blockReport_04() throws IOException {
|
|
|
+ final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
+ Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
|
+ DFSTestUtil.createFile(fs, filePath,
|
|
|
+ FILE_SIZE, REPL_FACTOR, rand.nextLong());
|
|
|
+
|
|
|
+
|
|
|
+ DataNode dn = cluster.getDataNodes().get(DN_N0);
|
|
|
+ // all blocks belong to the same file, hence same BP
|
|
|
+ String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
+
|
|
|
+ // Create a bogus new block which will not be present on the namenode.
|
|
|
+ ExtendedBlock b = new ExtendedBlock(
|
|
|
+ poolId, rand.nextLong(), 1024L, rand.nextLong());
|
|
|
+ dn.getFSDataset().createRbw(b);
|
|
|
+
|
|
|
+ DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn, poolId);
|
|
|
+ DatanodeCommand dnCmd =
|
|
|
+ cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
+ if(LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("Got the command: " + dnCmd);
|
|
|
+ }
|
|
|
+ printStats();
|
|
|
+
|
|
|
+ assertThat("Wrong number of corrupt blocks",
|
|
|
+ cluster.getNamesystem().getCorruptReplicaBlocks(), is(0L));
|
|
|
+ assertThat("Wrong number of PendingDeletion blocks",
|
|
|
+ cluster.getNamesystem().getPendingDeletionBlocks(), is(1L));
|
|
|
}
|
|
|
|
|
|
// Client requests new block from NN. The test corrupts this very block
|
|
@@ -331,10 +406,8 @@ public class TestBlockReport {
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = { new StorageBlockReport(
|
|
|
- new DatanodeStorage(dnR.getDatanodeUuid()),
|
|
|
- new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
|
|
|
- cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn, poolId);
|
|
|
+ cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
printStats();
|
|
|
assertEquals("Wrong number of PendingReplication Blocks",
|
|
|
0, cluster.getNamesystem().getUnderReplicatedBlocks());
|
|
@@ -382,9 +455,7 @@ public class TestBlockReport {
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = { new StorageBlockReport(
|
|
|
- new DatanodeStorage(dnR.getDatanodeUuid()),
|
|
|
- new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
|
|
|
+ StorageBlockReport[] report = getBlockReports(dn, poolId);
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
printStats();
|
|
|
assertEquals("Wrong number of Corrupted blocks",
|
|
@@ -407,7 +478,7 @@ public class TestBlockReport {
|
|
|
}
|
|
|
|
|
|
report[0] = new StorageBlockReport(
|
|
|
- new DatanodeStorage(dnR.getDatanodeUuid()),
|
|
|
+ report[0].getStorage(),
|
|
|
new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
printStats();
|
|
@@ -458,9 +529,7 @@ public class TestBlockReport {
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = { new StorageBlockReport(
|
|
|
- new DatanodeStorage(dnR.getDatanodeUuid()),
|
|
|
- new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
|
|
|
+ StorageBlockReport[] report = getBlockReports(dn, poolId);
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
printStats();
|
|
|
assertEquals("Wrong number of PendingReplication blocks",
|
|
@@ -506,9 +575,7 @@ public class TestBlockReport {
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = { new StorageBlockReport(
|
|
|
- new DatanodeStorage(dnR.getDatanodeUuid()),
|
|
|
- new BlockListAsLongs(blocks, null).getBlockListAsLongs()) };
|
|
|
+ StorageBlockReport[] report = getBlockReports(dn, poolId);
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
printStats();
|
|
|
assertEquals("Wrong number of PendingReplication blocks",
|