You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by ar...@apache.org on 2013/10/31 03:32:25 UTC

svn commit: r1537365 - in /hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/protocol/ src/main/java/org/apache/hadoop/hdfs/server/datanode/ src/test/java/org/apache/hadoop/hdfs/server/datanode/

Author: arp
Date: Thu Oct 31 02:32:25 2013
New Revision: 1537365

URL: http://svn.apache.org/r1537365
Log:
HDFS-5437. Fix TestBlockReport and TestBPOfferService failures.

Modified:
    hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-2832.txt
    hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
    hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
    hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
    hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
    hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-2832.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-2832.txt?rev=1537365&r1=1537364&r2=1537365&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-2832.txt (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-2832.txt Thu Oct 31 02:32:25 2013
@@ -54,3 +54,6 @@ IMPROVEMENTS:
 
     HDFS-5435. File append fails to initialize storageIDs. (Junping Du via
     Arpit Agarwal)
+
+    HDFS-5437. Fix TestBlockReport and TestBPOfferService failures. (Arpit
+    Agarwal)

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java?rev=1537365&r1=1537364&r2=1537365&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java Thu Oct 31 02:32:25 2013
@@ -19,7 +19,9 @@ package org.apache.hadoop.hdfs.protocol;
 
 import java.util.Iterator;
 import java.util.List;
+import java.util.Random;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -250,33 +252,28 @@ public class BlockListAsLongs implements
   }
 
   /**
-   * The block-id of the indexTh block
-   * @param index - the block whose block-id is desired
-   * @return the block-id
-   */
-  @Deprecated
-  public long getBlockId(final int index)  {
-    return blockId(index);
-  }
-  
-  /**
-   * The block-len of the indexTh block
-   * @param index - the block whose block-len is desired
-   * @return - the block-len
+   * Corrupt the generation stamp of the block with the given index.
+   * Not meant to be used outside of tests.
    */
-  @Deprecated
-  public long getBlockLen(final int index)  {
-    return blockLength(index);
+  @VisibleForTesting
+  public long corruptBlockGSForTesting(final int blockIndex, Random rand) {
+    long oldGS = blockList[index2BlockId(blockIndex) + 2];
+    while (blockList[index2BlockId(blockIndex) + 2] == oldGS) {
+      blockList[index2BlockId(blockIndex) + 2] = rand.nextInt();
+    }
+    return oldGS;
   }
 
   /**
-   * The generation stamp of the indexTh block
-   * @param index - the block whose block-len is desired
-   * @return - the generation stamp
+   * Corrupt the length of the block with the given index by truncation.
+   * Not meant to be used outside of tests.
    */
-  @Deprecated
-  public long getBlockGenStamp(final int index)  {
-    return blockGenerationStamp(index);
+  @VisibleForTesting
+  public long corruptBlockLengthForTesting(final int blockIndex, Random rand) {
+    long oldLength = blockList[index2BlockId(blockIndex) + 1];
+    blockList[index2BlockId(blockIndex) + 1] =
+        rand.nextInt((int) oldLength - 1);
+    return oldLength;
   }
   
   /**

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java?rev=1537365&r1=1537364&r2=1537365&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java Thu Oct 31 02:32:25 2013
@@ -60,11 +60,7 @@ public class LocatedBlock {
   private static final DatanodeInfo[] EMPTY_LOCS = new DatanodeInfo[0];
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {
-    this(b, locs, -1); // startOffset is unknown
-  }
-
-  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset) {
-    this(b, locs, startOffset, false);
+    this(b, locs, -1, false); // startOffset is unknown
   }
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs, long startOffset, 
@@ -76,6 +72,11 @@ public class LocatedBlock {
     this(b, storages, -1, false); // startOffset is unknown
   }
 
+  public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs,
+                      String[] storageIDs, StorageType[] storageTypes) {
+    this(b, locs, storageIDs, storageTypes, -1, false, EMPTY_LOCS);
+  }
+
   public LocatedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
       long startOffset, boolean corrupt) {
     this(b, DatanodeStorageInfo.toDatanodeInfos(storages),

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java?rev=1537365&r1=1537364&r2=1537365&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java Thu Oct 31 02:32:25 2013
@@ -252,7 +252,7 @@ class BPServiceActor implements Runnable
     // TODO: Corrupt flag is set to false for compatibility. We can probably
     // set it to true here.
     LocatedBlock[] blocks = {
-        new LocatedBlock(block, dnArr, uuids, types, -1, false, null) };
+        new LocatedBlock(block, dnArr, uuids, types) };
     
     try {
       bpNamenode.reportBadBlocks(blocks);  

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java?rev=1537365&r1=1537364&r2=1537365&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java Thu Oct 31 02:32:25 2013
@@ -1029,7 +1029,7 @@ public class SimulatedFSDataset implemen
 
   @Override
   public StorageReport[] getStorageReports(String bpid) {
-    throw new UnsupportedOperationException();
+    return new StorageReport[0];
   }
 
   @Override

Modified: hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java?rev=1537365&r1=1537364&r2=1537365&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java (original)
+++ hadoop/common/branches/HDFS-2832/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReport.java Thu Oct 31 02:32:25 2013
@@ -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();
+      BlockListAsLongs blockList = kvPair.getValue();
 
-      // 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;
+      // 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 =