Ver código fonte

HDFS-9766. TestDataNodeMetrics#testDataNodeTimeSpend fails intermittently. Contributed by Xiao Chen.

(cherry picked from commit 81fbc40f084eab85750cc7d4cc967d43d70d8333)
Akira Ajisaka 9 anos atrás
pai
commit
e840141726

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

@@ -2020,6 +2020,9 @@ Release 2.7.3 - UNRELEASED
 
     HDFS-9880. TestDatanodeRegistration fails occasionally (kihwal)
 
+    HDFS-9766. TestDataNodeMetrics#testDataNodeTimeSpend fails intermittently.
+    (Xiao Chen via aajisaka)
+
 Release 2.7.2 - 2016-01-25
 
   INCOMPATIBLE CHANGES

+ 28 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeMetrics.java

@@ -26,8 +26,10 @@ import static org.junit.Assert.*;
 import java.io.Closeable;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.List;
 
+import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
 
 import org.apache.commons.logging.Log;
@@ -47,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -255,38 +258,43 @@ public class TestDataNodeMetrics {
    * and reading causes totalReadTime to move.
    * @throws Exception
    */
-  @Test
+  @Test(timeout=60000)
   public void testDataNodeTimeSpend() throws Exception {
     Configuration conf = new HdfsConfiguration();
     SimulatedFSDataset.setFactory(conf);
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
     try {
-      FileSystem fs = cluster.getFileSystem();
+      final FileSystem fs = cluster.getFileSystem();
       List<DataNode> datanodes = cluster.getDataNodes();
       assertEquals(datanodes.size(), 1);
-      DataNode datanode = datanodes.get(0);
+      final DataNode datanode = datanodes.get(0);
       MetricsRecordBuilder rb = getMetrics(datanode.getMetrics().name());
       final long LONG_FILE_LEN = 1024 * 1024 * 10;
 
-      long startWriteValue = getLongCounter("TotalWriteTime", rb);
-      long startReadValue = getLongCounter("TotalReadTime", rb);
-
-      for (int x =0; x < 50; x++) {
-        DFSTestUtil.createFile(fs, new Path("/time.txt."+ x),
-                LONG_FILE_LEN, (short) 1, Time.monotonicNow());
-      }
-
-      for (int x =0; x < 50; x++) {
-        DFSTestUtil.readFile(fs, new Path("/time.txt." + x));
-      }
+      final long startWriteValue = getLongCounter("TotalWriteTime", rb);
+      final long startReadValue = getLongCounter("TotalReadTime", rb);
+      final AtomicInteger x = new AtomicInteger(0);
 
-      MetricsRecordBuilder rbNew = getMetrics(datanode.getMetrics().name());
-      long endWriteValue = getLongCounter("TotalWriteTime", rbNew);
-      long endReadValue = getLongCounter("TotalReadTime", rbNew);
       // Lets Metric system update latest metrics
-      Thread.sleep(100);
-      assertTrue(endReadValue > startReadValue);
-      assertTrue(endWriteValue > startWriteValue);
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          x.getAndIncrement();
+          try {
+            DFSTestUtil.createFile(fs, new Path("/time.txt." + x.get()),
+                LONG_FILE_LEN, (short) 1, Time.monotonicNow());
+            DFSTestUtil.readFile(fs, new Path("/time.txt." + x.get()));
+          } catch (IOException ioe) {
+            LOG.error("Caught IOException while ingesting DN metrics", ioe);
+            return false;
+          }
+          MetricsRecordBuilder rbNew = getMetrics(datanode.getMetrics().name());
+          final long endWriteValue = getLongCounter("TotalWriteTime", rbNew);
+          final long endReadValue = getLongCounter("TotalReadTime", rbNew);
+          return endWriteValue > startWriteValue
+              && endReadValue > startReadValue;
+        }
+      }, 30, 30000);
     } finally {
       if (cluster != null) {
         cluster.shutdown();