Browse Source

HDFS-10720. Fix intermittent test failure of TestDataNodeErasureCodingMetrics. Contributed by Rakesh R

Kai Zheng 8 years ago
parent
commit
e7e8aed208

+ 37 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeErasureCodingMetrics.java

@@ -39,12 +39,15 @@ import static org.apache.hadoop.test.MetricsAsserts.assertCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getLongCounter;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.util.Arrays;
 
 
 
 
 /**
 /**
@@ -125,39 +128,64 @@ public class TestDataNodeErasureCodingMetrics {
     DataNode workerDn = null;
     DataNode workerDn = null;
     DatanodeInfo[] locations = lastBlock.getLocations();
     DatanodeInfo[] locations = lastBlock.getLocations();
     assertEquals(locations.length, GROUPSIZE);
     assertEquals(locations.length, GROUPSIZE);
+
     // we have ONE extra datanode in addition to the GROUPSIZE datanodes, here
     // we have ONE extra datanode in addition to the GROUPSIZE datanodes, here
     // is to find the extra datanode that the reconstruction task will run on,
     // is to find the extra datanode that the reconstruction task will run on,
     // according to the current block placement logic for striped files.
     // according to the current block placement logic for striped files.
     // This can be improved later to be flexible regardless wherever the task
     // This can be improved later to be flexible regardless wherever the task
     // runs.
     // runs.
-    for (DataNode dn: cluster.getDataNodes()) {
+    for (DataNode dn : cluster.getDataNodes()) {
       boolean appear = false;
       boolean appear = false;
-      for (DatanodeInfo info: locations) {
+      for (DatanodeInfo info : locations) {
         if (dn.getDatanodeUuid().equals(info.getDatanodeUuid())) {
         if (dn.getDatanodeUuid().equals(info.getDatanodeUuid())) {
           appear = true;
           appear = true;
           break;
           break;
         }
         }
       }
       }
-      if(!appear) {
+      if (!appear) {
         workerDn = dn;
         workerDn = dn;
         break;
         break;
       }
       }
     }
     }
-    byte[] indices = lastBlock.getBlockIndices();
-    //corrupt the first block
-    DataNode toCorruptDn = cluster.getDataNodes().get(indices[0]);
+    // Get a datanode from the block locations.
+    LOG.info("Block locations: " + Arrays.asList(locations));
+    LOG.info("Erasure coding worker datanode: " + workerDn);
+    assertNotNull("Failed to find a worker datanode", workerDn);
+
+    DataNode toCorruptDn = cluster.getDataNode(locations[0].getIpcPort());
+    LOG.info("Datanode to be corrupted: " + toCorruptDn);
+    assertNotNull("Failed to find a datanode to be corrupted", toCorruptDn);
     toCorruptDn.shutdown();
     toCorruptDn.shutdown();
     setDataNodeDead(toCorruptDn.getDatanodeId());
     setDataNodeDead(toCorruptDn.getDatanodeId());
     DFSTestUtil.waitForDatanodeState(cluster, toCorruptDn.getDatanodeUuid(),
     DFSTestUtil.waitForDatanodeState(cluster, toCorruptDn.getDatanodeUuid(),
-        false, 10000 );
-    final BlockManager bm = cluster.getNamesystem().getBlockManager();
-    BlockManagerTestUtil.getComputedDatanodeWork(bm);
+        false, 10000);
+
+    int workCount = getComputedDatanodeWork();
+    assertTrue("Wrongly computed block reconstruction work", workCount > 0);
     cluster.triggerHeartbeats();
     cluster.triggerHeartbeats();
     StripedFileTestUtil.waitForReconstructionFinished(file, fs, GROUPSIZE);
     StripedFileTestUtil.waitForReconstructionFinished(file, fs, GROUPSIZE);
 
 
     return workerDn;
     return workerDn;
   }
   }
 
 
+  private int getComputedDatanodeWork()
+      throws IOException, InterruptedException {
+    final BlockManager bm = cluster.getNamesystem().getBlockManager();
+    // Giving a grace period to compute datanode work.
+    int workCount = 0;
+    int retries = 20;
+    while (retries > 0) {
+      workCount = BlockManagerTestUtil.getComputedDatanodeWork(bm);
+      if (workCount > 0) {
+        break;
+      }
+      retries--;
+      Thread.sleep(500);
+    }
+    LOG.info("Computed datanode work: " + workCount + ", retries: " + retries);
+    return workCount;
+  }
+
   private void setDataNodeDead(DatanodeID dnID) throws IOException {
   private void setDataNodeDead(DatanodeID dnID) throws IOException {
     DatanodeDescriptor dnd =
     DatanodeDescriptor dnd =
         NameNodeAdapter.getDatanode(cluster.getNamesystem(), dnID);
         NameNodeAdapter.getDatanode(cluster.getNamesystem(), dnID);