소스 검색

HDFS-11445. FSCK shows overall health stauts as corrupt even one replica is corrupt. Contributed by Brahma Reddy Battula.

(cherry picked from commit 7bad748091b6a5f90e919c781a915e13000e1be6)
Brahma Reddy Battula 8 년 전
부모
커밋
4f5846f1e3

+ 7 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -25,7 +25,6 @@ import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.LightWeightGSet;
 
 import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID;
@@ -381,28 +380,25 @@ public abstract class BlockInfo extends Block
    * Process the recorded replicas. When about to commit or finish the
    * pipeline recovery sort out bad replicas.
    * @param genStamp  The final generation stamp for the block.
+   * @return staleReplica's List.
    */
-  public void setGenerationStampAndVerifyReplicas(long genStamp) {
+  public List<ReplicaUnderConstruction> setGenerationStampAndVerifyReplicas(
+      long genStamp) {
     Preconditions.checkState(uc != null && !isComplete());
     // Set the generation stamp for the block.
     setGenerationStamp(genStamp);
 
-    // Remove the replicas with wrong gen stamp
-    List<ReplicaUnderConstruction> staleReplicas = uc.getStaleReplicas(genStamp);
-    for (ReplicaUnderConstruction r : staleReplicas) {
-      r.getExpectedStorageLocation().removeBlock(this);
-      NameNode.blockStateChangeLog.debug("BLOCK* Removing stale replica {}"
-          + " of {}", r, Block.toString(r));
-    }
+    return uc.getStaleReplicas(genStamp);
   }
 
   /**
    * Commit block's length and generation stamp as reported by the client.
    * Set block state to {@link BlockUCState#COMMITTED}.
    * @param block - contains client reported block length and generation
+   * @return staleReplica's List.
    * @throws IOException if block ids are inconsistent.
    */
-  void commitBlock(Block block) throws IOException {
+  List<ReplicaUnderConstruction> commitBlock(Block block) throws IOException {
     if (getBlockId() != block.getBlockId()) {
       throw new IOException("Trying to commit inconsistent block: id = "
           + block.getBlockId() + ", expected id = " + getBlockId());
@@ -411,6 +407,6 @@ public abstract class BlockInfo extends Block
     uc.commit();
     this.setNumBytes(block.getNumBytes());
     // Sort out invalid replicas.
-    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
+    return setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
   }
 }

+ 24 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -699,7 +699,7 @@ public class BlockManager implements BlockStatsMXBean {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private static boolean commitBlock(final BlockInfo block,
+  private boolean commitBlock(final BlockInfo block,
       final Block commitBlock) throws IOException {
     if (block.getBlockUCState() == BlockUCState.COMMITTED)
       return false;
@@ -710,7 +710,9 @@ public class BlockManager implements BlockStatsMXBean {
       throw new IOException("Commit block with mismatching GS. NN has " +
         block + ", client submits " + commitBlock);
     }
-    block.commitBlock(commitBlock);
+    List<ReplicaUnderConstruction> staleReplicas =
+        block.commitBlock(commitBlock);
+    removeStaleReplicas(staleReplicas, block);
     return true;
   }
   
@@ -836,7 +838,8 @@ public class BlockManager implements BlockStatsMXBean {
    * when tailing edit logs as a Standby.
    */
   public void forceCompleteBlock(final BlockInfo block) throws IOException {
-    block.commitBlock(block);
+    List<ReplicaUnderConstruction> staleReplicas = block.commitBlock(block);
+    removeStaleReplicas(staleReplicas, block);
     completeBlock(block, null, true);
   }
 
@@ -3173,6 +3176,17 @@ public class BlockManager implements BlockStatsMXBean {
     }
   }
 
+  private void removeStaleReplicas(List<ReplicaUnderConstruction> staleReplicas,
+      BlockInfo block) {
+    for (ReplicaUnderConstruction r : staleReplicas) {
+      removeStoredBlock(block,
+          r.getExpectedStorageLocation().getDatanodeDescriptor());
+      NameNode.blockStateChangeLog
+          .debug("BLOCK* Removing stale replica {}" + " of {}", r,
+              Block.toString(r));
+    }
+  }
+
   /**
    * Get all valid locations of the block & add the block to results
    * return the length of the added block; 0 if the block is not added
@@ -3509,6 +3523,13 @@ public class BlockManager implements BlockStatsMXBean {
     return blocksMap.getStoredBlock(block);
   }
 
+  public void updateLastBlock(BlockInfo lastBlock, ExtendedBlock newBlock) {
+    lastBlock.setNumBytes(newBlock.getNumBytes());
+    List<ReplicaUnderConstruction> staleReplicas = lastBlock
+        .setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
+    removeStaleReplicas(staleReplicas, lastBlock);
+  }
+
   /** updates a block in under replication queue */
   private void updateNeededReplications(final BlockInfo block,
       final int curReplicasDelta, int expectedReplicasDelta) {

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -5620,8 +5620,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // Update old block with the new generation stamp and new length
-    blockinfo.setNumBytes(newBlock.getNumBytes());
-    blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp());
+    blockManager.updateLastBlock(blockinfo, newBlock);
 
     // find the DatanodeDescriptor objects
     final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager()

+ 32 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -77,6 +77,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -97,6 +98,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.hdfs.tools.DFSck;
 import org.apache.hadoop.hdfs.util.HostsFileWriter;
@@ -1789,4 +1791,34 @@ public class TestFsck {
       }
     }
   }
+
+  @Test(timeout = 300000)
+  public void testFsckCorruptWhenOneReplicaIsCorrupt()
+      throws Exception {
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(2)
+        .build()) {
+      cluster.waitActive();
+      FileSystem fs = HATestUtil.configureFailoverFs(cluster, conf);
+      cluster.transitionToActive(0);
+      String filePath = "/appendTest";
+      Path fileName = new Path(filePath);
+      DFSTestUtil.createFile(fs, fileName, 512, (short) 2, 0);
+      DFSTestUtil.waitReplication(fs, fileName, (short) 2);
+      Assert.assertTrue("File not created", fs.exists(fileName));
+      cluster.getDataNodes().get(1).shutdown();
+      DFSTestUtil.appendFile(fs, fileName, "appendCorruptBlock");
+      cluster.restartDataNode(1, true);
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override public Boolean get() {
+          return (
+              cluster.getNameNode(0).getNamesystem().getCorruptReplicaBlocks()
+                  > 0);
+        }
+      }, 100, 5000);
+
+      DFSTestUtil.appendFile(fs, fileName, "appendCorruptBlock");
+      runFsck(cluster.getConfiguration(0), 0, true, "/");
+    }
+  }
 }