|
@@ -116,52 +116,45 @@ public class TestBlockReport {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
|
|
|
- private static StorageBlockReport[] getBlockReports(DataNode dn, String bpid) {
|
|
|
+ // Generate a block report, optionally corrupting the generation
|
|
|
+ // stamp and/or length of one block.
|
|
|
+ private static StorageBlockReport[] getBlockReports(
|
|
|
+ DataNode dn, String bpid, boolean corruptOneBlockGs,
|
|
|
+ boolean corruptOneBlockLen) {
|
|
|
Map<String, BlockListAsLongs> perVolumeBlockLists =
|
|
|
dn.getFSDataset().getBlockReports(bpid);
|
|
|
|
|
|
// Send block report
|
|
|
StorageBlockReport[] reports =
|
|
|
new StorageBlockReport[perVolumeBlockLists.size()];
|
|
|
+ boolean corruptedGs = false;
|
|
|
+ boolean corruptedLen = false;
|
|
|
|
|
|
- int i = 0;
|
|
|
+ int reportIndex = 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;
|
|
|
+ BlockListAsLongs blockList = kvPair.getValue();
|
|
|
+
|
|
|
+ // Walk the list of blocks until we find one each to corrupt the
|
|
|
+ // generation stamp and length, if so requested.
|
|
|
+ for (int i = 0; i < blockList.getNumberOfBlocks(); ++i) {
|
|
|
+ if (corruptOneBlockGs && !corruptedGs) {
|
|
|
+ blockList.corruptBlockGSForTesting(i, rand);
|
|
|
+ LOG.info("Corrupted the GS for block ID " + i);
|
|
|
+ corruptedGs = true;
|
|
|
+ } else if (corruptOneBlockLen && !corruptedLen) {
|
|
|
+ blockList.corruptBlockLengthForTesting(i, rand);
|
|
|
+ LOG.info("Corrupted the length for block ID " + i);
|
|
|
+ corruptedLen = true;
|
|
|
+ } else {
|
|
|
+ break;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
// Dummy DatanodeStorage object just for sending the block report.
|
|
|
DatanodeStorage dnStorage = new DatanodeStorage(storageID);
|
|
|
- reports[i++] = new StorageBlockReport(dnStorage, blockList);
|
|
|
+ reports[reportIndex++] =
|
|
|
+ new StorageBlockReport(dnStorage, blockList.getBlockListAsLongs());
|
|
|
}
|
|
|
|
|
|
return reports;
|
|
@@ -207,7 +200,7 @@ public class TestBlockReport {
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N0);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] reports = getBlockReports(dn, poolId);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
|
|
|
List<LocatedBlock> blocksAfterReport =
|
|
@@ -289,7 +282,7 @@ public class TestBlockReport {
|
|
|
// all blocks belong to the same file, hence same BP
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn0.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] reports = getBlockReports(dn0, poolId);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn0, poolId, false, false);
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
|
|
|
BlockManagerTestUtil.getComputedDatanodeWork(cluster.getNamesystem()
|
|
@@ -315,14 +308,13 @@ public class TestBlockReport {
|
|
|
public void blockReport_03() 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());
|
|
|
+ ArrayList<Block> blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath);
|
|
|
|
|
|
// 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[] reports = getBlockReportsCorruptSingleBlockGS(dn, poolId);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
|
|
|
DatanodeCommand dnCmd =
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
if(LOG.isDebugEnabled()) {
|
|
@@ -362,7 +354,7 @@ public class TestBlockReport {
|
|
|
dn.getFSDataset().createRbw(b);
|
|
|
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] reports = getBlockReports(dn, poolId);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
|
DatanodeCommand dnCmd =
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
if(LOG.isDebugEnabled()) {
|
|
@@ -376,15 +368,6 @@ public class TestBlockReport {
|
|
|
cluster.getNamesystem().getPendingDeletionBlocks(), is(1L));
|
|
|
}
|
|
|
|
|
|
- // Client requests new block from NN. The test corrupts this very block
|
|
|
- // and forces new block report.
|
|
|
- // The test case isn't specific for BlockReport because it relies on
|
|
|
- // BlockScanner which is out of scope of this test
|
|
|
- // Keeping the name to be in synch with the test plan
|
|
|
- //
|
|
|
- public void blockReport_05() {
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Test creates a file and closes it.
|
|
|
* The second datanode is started in the cluster.
|
|
@@ -399,14 +382,14 @@ public class TestBlockReport {
|
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
|
final int DN_N1 = DN_N0 + 1;
|
|
|
|
|
|
- ArrayList<Block> blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath);
|
|
|
+ writeFile(METHOD_NAME, FILE_SIZE, filePath);
|
|
|
startDNandWait(filePath, true);
|
|
|
|
|
|
- // all blocks belong to the same file, hence same BP
|
|
|
+ // all blocks belong to the same file, hence same BP
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] reports = getBlockReports(dn, poolId);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn, poolId, false, false);
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
printStats();
|
|
|
assertEquals("Wrong number of PendingReplication Blocks",
|
|
@@ -427,66 +410,40 @@ public class TestBlockReport {
|
|
|
* @throws IOException in case of an error
|
|
|
*/
|
|
|
@Test
|
|
|
- // Currently this test is failing as expected 'cause the correct behavior is
|
|
|
- // not yet implemented (9/15/09)
|
|
|
public void blockReport_07() throws Exception {
|
|
|
final String METHOD_NAME = GenericTestUtils.getMethodName();
|
|
|
Path filePath = new Path("/" + METHOD_NAME + ".dat");
|
|
|
final int DN_N1 = DN_N0 + 1;
|
|
|
|
|
|
// write file and start second node to be "older" than the original
|
|
|
- ArrayList<Block> blocks = writeFile(METHOD_NAME, FILE_SIZE, filePath);
|
|
|
+ writeFile(METHOD_NAME, FILE_SIZE, filePath);
|
|
|
startDNandWait(filePath, true);
|
|
|
|
|
|
- int randIndex = rand.nextInt(blocks.size());
|
|
|
- // Get a block and screw its GS
|
|
|
- Block corruptedBlock = blocks.get(randIndex);
|
|
|
- String secondNode = cluster.getDataNodes().get(DN_N1).getDatanodeId().getDatanodeUuid();
|
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Working with " + secondNode);
|
|
|
- LOG.debug("BlockGS before " + blocks.get(randIndex).getGenerationStamp());
|
|
|
- }
|
|
|
- corruptBlockGS(corruptedBlock);
|
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("BlockGS after " + blocks.get(randIndex).getGenerationStamp());
|
|
|
- LOG.debug("Done corrupting GS of " + corruptedBlock.getBlockName());
|
|
|
- }
|
|
|
// all blocks belong to the same file, hence same BP
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = getBlockReports(dn, poolId);
|
|
|
- cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
+ StorageBlockReport[] reports = getBlockReports(dn, poolId, true, false);
|
|
|
+ cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
printStats();
|
|
|
- assertEquals("Wrong number of Corrupted blocks",
|
|
|
- 1, cluster.getNamesystem().getCorruptReplicaBlocks() +
|
|
|
-// the following might have to be added into the equation if
|
|
|
-// the same block could be in two different states at the same time
|
|
|
-// and then the expected number of has to be changed to '2'
|
|
|
-// cluster.getNamesystem().getPendingReplicationBlocks() +
|
|
|
- cluster.getNamesystem().getPendingDeletionBlocks());
|
|
|
-
|
|
|
- // Get another block and screw its length to be less than original
|
|
|
- if (randIndex == 0)
|
|
|
- randIndex++;
|
|
|
- else
|
|
|
- randIndex--;
|
|
|
- corruptedBlock = blocks.get(randIndex);
|
|
|
- corruptBlockLen(corruptedBlock);
|
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Done corrupting length of " + corruptedBlock.getBlockName());
|
|
|
- }
|
|
|
-
|
|
|
- report[0] = new StorageBlockReport(
|
|
|
- report[0].getStorage(),
|
|
|
- new BlockListAsLongs(blocks, null).getBlockListAsLongs());
|
|
|
- cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
+
|
|
|
+ assertThat("Wrong number of corrupt blocks",
|
|
|
+ cluster.getNamesystem().getCorruptReplicaBlocks(), is(1L));
|
|
|
+ assertThat("Wrong number of PendingDeletion blocks",
|
|
|
+ cluster.getNamesystem().getPendingDeletionBlocks(), is(0L));
|
|
|
+ assertThat("Wrong number of PendingReplication blocks",
|
|
|
+ cluster.getNamesystem().getPendingReplicationBlocks(), is(0L));
|
|
|
+
|
|
|
+ reports = getBlockReports(dn, poolId, true, true);
|
|
|
+ cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
|
|
|
printStats();
|
|
|
|
|
|
- assertEquals("Wrong number of Corrupted blocks",
|
|
|
- 2, cluster.getNamesystem().getCorruptReplicaBlocks() +
|
|
|
- cluster.getNamesystem().getPendingReplicationBlocks() +
|
|
|
- cluster.getNamesystem().getPendingDeletionBlocks());
|
|
|
+ assertThat("Wrong number of corrupt blocks",
|
|
|
+ cluster.getNamesystem().getCorruptReplicaBlocks(), is(2L));
|
|
|
+ assertThat("Wrong number of PendingDeletion blocks",
|
|
|
+ cluster.getNamesystem().getPendingDeletionBlocks(), is(0L));
|
|
|
+ assertThat("Wrong number of PendingReplication blocks",
|
|
|
+ cluster.getNamesystem().getPendingReplicationBlocks(), is(0L));
|
|
|
|
|
|
printStats();
|
|
|
|
|
@@ -529,7 +486,7 @@ public class TestBlockReport {
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = getBlockReports(dn, poolId);
|
|
|
+ StorageBlockReport[] report = getBlockReports(dn, poolId, false, false);
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
printStats();
|
|
|
assertEquals("Wrong number of PendingReplication blocks",
|
|
@@ -560,14 +517,11 @@ public class TestBlockReport {
|
|
|
// write file and start second node to be "older" than the original
|
|
|
|
|
|
try {
|
|
|
- ArrayList<Block> blocks =
|
|
|
- writeFile(METHOD_NAME, 12 * bytesChkSum, filePath);
|
|
|
+ writeFile(METHOD_NAME, 12 * bytesChkSum, filePath);
|
|
|
|
|
|
Block bl = findBlock(filePath, 12 * bytesChkSum);
|
|
|
BlockChecker bc = new BlockChecker(filePath);
|
|
|
bc.start();
|
|
|
- corruptBlockGS(bl);
|
|
|
- corruptBlockLen(bl);
|
|
|
|
|
|
waitForTempReplica(bl, DN_N1);
|
|
|
|
|
@@ -575,7 +529,7 @@ public class TestBlockReport {
|
|
|
DataNode dn = cluster.getDataNodes().get(DN_N1);
|
|
|
String poolId = cluster.getNamesystem().getBlockPoolId();
|
|
|
DatanodeRegistration dnR = dn.getDNRegistrationForBP(poolId);
|
|
|
- StorageBlockReport[] report = getBlockReports(dn, poolId);
|
|
|
+ StorageBlockReport[] report = getBlockReports(dn, poolId, true, true);
|
|
|
cluster.getNameNodeRpc().blockReport(dnR, poolId, report);
|
|
|
printStats();
|
|
|
assertEquals("Wrong number of PendingReplication blocks",
|
|
@@ -851,38 +805,6 @@ public class TestBlockReport {
|
|
|
((Log4JLogger) TestBlockReport.LOG).getLogger().setLevel(Level.ALL);
|
|
|
}
|
|
|
|
|
|
- private void corruptBlockLen(final Block block)
|
|
|
- throws IOException {
|
|
|
- if (block == null) {
|
|
|
- throw new IOException("Block isn't suppose to be null");
|
|
|
- }
|
|
|
- long oldLen = block.getNumBytes();
|
|
|
- long newLen = oldLen - rand.nextLong();
|
|
|
- assertTrue("Old and new length shouldn't be the same",
|
|
|
- block.getNumBytes() != newLen);
|
|
|
- block.setNumBytes(newLen);
|
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Length of " + block.getBlockName() +
|
|
|
- " is changed to " + newLen + " from " + oldLen);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private void corruptBlockGS(final Block block)
|
|
|
- throws IOException {
|
|
|
- if (block == null) {
|
|
|
- throw new IOException("Block isn't suppose to be null");
|
|
|
- }
|
|
|
- long oldGS = block.getGenerationStamp();
|
|
|
- long newGS = oldGS - rand.nextLong();
|
|
|
- assertTrue("Old and new GS shouldn't be the same",
|
|
|
- block.getGenerationStamp() != newGS);
|
|
|
- block.setGenerationStamp(newGS);
|
|
|
- if(LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Generation stamp of " + block.getBlockName() +
|
|
|
- " is changed to " + block.getGenerationStamp() + " from " + oldGS);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
private Block findBlock(Path path, long size) throws IOException {
|
|
|
Block ret;
|
|
|
List<LocatedBlock> lbs =
|