Explorar o código

HDFS-13819. TestDirectoryScanner#testDirectoryScannerInFederatedCluster is flaky

Change-Id: I1cea6e67fcec72702ad202775dee3373261ac5cd
Daniel Templeton %!s(int64=6) %!d(string=hai) anos
pai
achega
4a5006b1d0

+ 32 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -40,6 +40,7 @@ import java.util.Random;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.io.FileUtils;
@@ -312,18 +313,29 @@ public class TestDirectoryScanner {
     return id;
   }
 
-  private void scan(long totalBlocks, int diffsize, long missingMetaFile, long missingBlockFile,
-      long missingMemoryBlocks, long mismatchBlocks) throws IOException {
+  private void scan(long totalBlocks, int diffsize, long missingMetaFile,
+      long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks)
+      throws IOException, InterruptedException, TimeoutException {
     scan(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
          missingMemoryBlocks, mismatchBlocks, 0);
   }
 
   private void scan(long totalBlocks, int diffsize, long missingMetaFile,
       long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks,
-      long duplicateBlocks) throws IOException {
+      long duplicateBlocks)
+      throws IOException, InterruptedException, TimeoutException {
     scanner.reconcile();
-    verifyStats(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
-        missingMemoryBlocks, mismatchBlocks, duplicateBlocks);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        verifyStats(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
+            missingMemoryBlocks, mismatchBlocks, duplicateBlocks);
+      } catch (AssertionError ex) {
+        return false;
+      }
+
+      return true;
+    }, 50, 2000);
   }
 
   private void verifyStats(long totalBlocks, int diffsize, long missingMetaFile,
@@ -785,7 +797,8 @@ public class TestDirectoryScanner {
     }
   }
 
-  private float runThrottleTest(int blocks) throws IOException {
+  private float runThrottleTest(int blocks)
+      throws IOException, InterruptedException, TimeoutException {
     scanner.setRetainDiffs(true);
     scan(blocks, 0, 0, 0, 0, 0);
     scanner.shutdown();
@@ -1069,10 +1082,19 @@ public class TestDirectoryScanner {
       scanner.setRetainDiffs(true);
       scanner.reconcile();
       //Check blocks in corresponding BP
-      bpid = cluster.getNamesystem(1).getBlockPoolId();
-      verifyStats(bp1Files, 0, 0, 0, 0, 0, 0);
-      bpid = cluster.getNamesystem(3).getBlockPoolId();
-      verifyStats(bp2Files, 0, 0, 0, 0, 0, 0);
+
+      GenericTestUtils.waitFor(() -> {
+        try {
+          bpid = cluster.getNamesystem(1).getBlockPoolId();
+          verifyStats(bp1Files, 0, 0, 0, 0, 0, 0);
+          bpid = cluster.getNamesystem(3).getBlockPoolId();
+          verifyStats(bp2Files, 0, 0, 0, 0, 0, 0);
+        } catch (AssertionError ex) {
+          return false;
+        }
+
+        return true;
+      }, 50, 2000);
     } finally {
       if (scanner != null) {
         scanner.shutdown();