Browse Source

HDFS-3119. Overreplicated block is not deleted even after the replication factor is reduced after sync follwed by closing that file. Contributed by Ashish Singhi.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1311391 13f79535-47bb-0310-9956-ffa450edef68
Uma Maheswara Rao G 13 years ago
parent
commit
75f105d0b4

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -339,6 +339,10 @@ Release 2.0.0 - UNRELEASED
     HDFS-3214. InterDatanodeProtocolServerSideTranslatorPB doesn't handle
     null response from initReplicaRecovery (todd)
 
+    HDFS-3119. Overreplicated block is not deleted even after the replication 
+    factor is reduced after sync follwed by closing that file. (Ashish Singhi 
+    via umamahesh)
+
   BREAKDOWN OF HDFS-1623 SUBTASKS
 
     HDFS-2179. Add fencing framework and mechanisms for NameNode HA. (todd)

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java

@@ -2769,7 +2769,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
     }
   }
 
-  public void checkReplication(Block block, int numExpectedReplicas) {
+  public void checkReplication(Block block, short numExpectedReplicas) {
     // filter out containingNodes that are marked for decommission.
     NumberReplicas number = countNodes(block);
     if (isNeededReplication(block, numExpectedReplicas, number.liveReplicas())) { 
@@ -2777,6 +2777,10 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block
                              number.liveReplicas(),
                              number.decommissionedReplicas(),
                              numExpectedReplicas);
+      return;
+    }
+    if (number.liveReplicas() > numExpectedReplicas) {
+      processOverReplicatedBlock(block, numExpectedReplicas, null, null);
     }
   }
 

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

@@ -2120,10 +2120,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /** 
    * Check all blocks of a file. If any blocks are lower than their intended
-   * replication factor, then insert them into neededReplication
+   * replication factor, then insert them into neededReplication and if 
+   * the blocks are more than the intended replication factor then insert 
+   * them into invalidateBlocks.
    */
   private void checkReplicationFactor(INodeFile file) {
-    int numExpectedReplicas = file.getReplication();
+    short numExpectedReplicas = file.getReplication();
     Block[] pendingBlocks = file.getBlocks();
     int nrBlocks = pendingBlocks.length;
     for (int i = 0; i < nrBlocks; i++) {

+ 31 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java

@@ -17,12 +17,13 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.junit.Assert.*;
 import java.io.File;
 import java.io.IOException;
 
-import junit.framework.TestCase;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -36,13 +37,15 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.junit.Test;
 
-public class TestOverReplicatedBlocks extends TestCase {
+public class TestOverReplicatedBlocks {
   /** Test processOverReplicatedBlock can handle corrupt replicas fine.
    * It make sure that it won't treat corrupt replicas as valid ones 
    * thus prevents NN deleting valid replicas but keeping
    * corrupt ones.
    */
+  @Test
   public void testProcesOverReplicateBlock() throws IOException {
     Configuration conf = new HdfsConfiguration();
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000L);
@@ -113,4 +116,30 @@ public class TestOverReplicatedBlocks extends TestCase {
       cluster.shutdown();
     }
   }
+  /**
+   * Test over replicated block should get invalidated when decreasing the
+   * replication for a partial block.
+   */
+  @Test
+  public void testInvalidateOverReplicatedBlock() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+        .build();
+    try {
+      final FSNamesystem namesystem = cluster.getNamesystem();
+      final BlockManager bm = namesystem.getBlockManager();
+      FileSystem fs = cluster.getFileSystem();
+      Path p = new Path(MiniDFSCluster.getBaseDirectory(), "/foo1");
+      FSDataOutputStream out = fs.create(p, (short) 2);
+      out.writeBytes("HDFS-3119: " + p);
+      out.hsync();
+      fs.setReplication(p, (short) 1);
+      out.close();
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, p);
+      assertEquals("Expected only one live replica for the block", 1, bm
+          .countNodes(block.getLocalBlock()).liveReplicas());
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }