Explorar el Código

HDFS-7886. Fix TestFileTruncate falures. Contributed by Plamen Jeliazkov and Konstantin Shvachko.

Konstantin V Shvachko hace 10 años
padre
commit
ad4d704fbb

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

@@ -839,6 +839,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-7903. Cannot recover block after truncate and delete snapshot.
     (Plamen Jeliazkov via shv)
 
+    HDFS-7886. Fix TestFileTruncate falures. (Plamen Jeliazkov and shv)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

+ 40 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -78,9 +78,12 @@ import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Util;
@@ -2015,7 +2018,23 @@ public class MiniDFSCluster {
    */
   public synchronized boolean restartDataNode(int i, boolean keepPort)
       throws IOException {
-    DataNodeProperties dnprop = stopDataNode(i);
+    return restartDataNode(i, keepPort, false);
+  }
+
+  /**
+   * Restart a particular DataNode.
+   * @param idn index of the DataNode
+   * @param keepPort true if should restart on the same port
+   * @param expireOnNN true if NameNode should expire the DataNode heartbeat
+   * @return
+   * @throws IOException
+   */
+  public synchronized boolean restartDataNode(
+      int idn, boolean keepPort, boolean expireOnNN) throws IOException {
+    DataNodeProperties dnprop = stopDataNode(idn);
+    if(expireOnNN) {
+      setDataNodeDead(dnprop.datanode.getDatanodeId());
+    }
     if (dnprop == null) {
       return false;
     } else {
@@ -2023,6 +2042,24 @@ public class MiniDFSCluster {
     }
   }
 
+  /**
+   * Expire a DataNode heartbeat on the NameNode
+   * @param dnId
+   * @throws IOException
+   */
+  public void setDataNodeDead(DatanodeID dnId) throws IOException {
+    DatanodeDescriptor dnd =
+        NameNodeAdapter.getDatanode(getNamesystem(), dnId);
+    dnd.setLastUpdate(0L);
+    BlockManagerTestUtil.checkHeartbeat(getNamesystem().getBlockManager());
+  }
+
+  public void setDataNodesDead() throws IOException {
+    for (DataNodeProperties dnp : dataNodes) {
+      setDataNodeDead(dnp.datanode.getDatanodeId());
+    }
+  }
+
   /*
    * Restart all datanodes, on the same ports if keepPort is true
    */
@@ -2278,8 +2315,8 @@ public class MiniDFSCluster {
     // make sure all datanodes have sent first heartbeat to namenode,
     // using (capacity == 0) as proxy.
     for (DatanodeInfo dn : dnInfo) {
-      if (dn.getCapacity() == 0) {
-        LOG.info("dn.getCapacity() == 0");
+      if (dn.getCapacity() == 0 || dn.getLastUpdate() <= 0) {
+        LOG.info("No heartbeat from DataNode: " + dn.toString());
         return true;
       }
     }

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java

@@ -679,10 +679,10 @@ public class TestFileTruncate {
       boolean isReady = fs.truncate(p, newLength);
       assertFalse(isReady);
     } finally {
-      cluster.restartDataNode(dn);
+      cluster.restartDataNode(dn, true, true);
       cluster.waitActive();
-      cluster.triggerBlockReports();
     }
+    checkBlockRecovery(p);
 
     LocatedBlock newBlock = getLocatedBlocks(p).getLastLocatedBlock();
     /*
@@ -699,7 +699,6 @@ public class TestFileTruncate {
     assertEquals(newBlock.getBlock().getGenerationStamp(),
         oldBlock.getBlock().getGenerationStamp() + 1);
 
-    checkBlockRecovery(p);
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
     // Old replica is disregarded and replaced with the truncated one
@@ -741,10 +740,10 @@ public class TestFileTruncate {
       boolean isReady = fs.truncate(p, newLength);
       assertFalse(isReady);
     } finally {
-      cluster.restartDataNode(dn);
+      cluster.restartDataNode(dn, true, true);
       cluster.waitActive();
-      cluster.triggerBlockReports();
     }
+    checkBlockRecovery(p);
 
     LocatedBlock newBlock = getLocatedBlocks(p).getLastLocatedBlock();
     /*
@@ -757,7 +756,6 @@ public class TestFileTruncate {
     assertEquals(newBlock.getBlock().getGenerationStamp(),
         oldBlock.getBlock().getGenerationStamp() + 1);
 
-    checkBlockRecovery(p);
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
     // New block is replicated to dn1
@@ -800,9 +798,10 @@ public class TestFileTruncate {
     boolean isReady = fs.truncate(p, newLength);
     assertFalse(isReady);
 
-    cluster.restartDataNode(dn0);
-    cluster.restartDataNode(dn1);
+    cluster.restartDataNode(dn0, true, true);
+    cluster.restartDataNode(dn1, true, true);
     cluster.waitActive();
+    checkBlockRecovery(p);
     cluster.triggerBlockReports();
 
     LocatedBlock newBlock = getLocatedBlocks(p).getLastLocatedBlock();
@@ -815,7 +814,6 @@ public class TestFileTruncate {
     assertEquals(newBlock.getBlock().getGenerationStamp(),
         oldBlock.getBlock().getGenerationStamp() + 1);
 
-    checkBlockRecovery(p);
     // Wait replicas come to 3
     DFSTestUtil.waitReplication(fs, p, REPLICATION);
     // Old replica is disregarded and replaced with the truncated one on dn0
@@ -859,6 +857,7 @@ public class TestFileTruncate {
     assertFalse(isReady);
 
     cluster.shutdownDataNodes();
+    cluster.setDataNodesDead();
     try {
       for(int i = 0; i < SUCCESS_ATTEMPTS && cluster.isDataNodeUp(); i++) {
         Thread.sleep(SLEEP);
@@ -871,6 +870,7 @@ public class TestFileTruncate {
           StartupOption.REGULAR, null);
       cluster.waitActive();
     }
+    checkBlockRecovery(p);
 
     fs.delete(parent, true);
   }