Parcourir la source

HDFS-15988. Stabilise HDFS Pre-Commit. (#2860). Contributed by Ayush Saxena.

Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
Ayush Saxena il y a 4 ans
Parent
commit
fcd4140e5f

+ 2 - 1
dev-support/bin/hadoop.sh

@@ -355,6 +355,7 @@ function personality_modules
       fi
     ;;
     unit)
+      extra="-Dsurefire.rerunFailingTestsCount=2"
       if [[ "${BUILDMODE}" = full ]]; then
         ordering=mvnsrc
       elif [[ "${CHANGED_MODULES[*]}" =~ \. ]]; then
@@ -363,7 +364,7 @@ function personality_modules
 
       if [[ ${TEST_PARALLEL} = "true" ]] ; then
         if hadoop_test_parallel; then
-          extra="-Pparallel-tests"
+          extra="${extra} -Pparallel-tests"
           if [[ -n ${TEST_THREADS:-} ]]; then
             extra="${extra} -DtestsThreadCount=${TEST_THREADS}"
           fi

+ 1 - 1
dev-support/docker/Dockerfile

@@ -179,7 +179,7 @@ RUN mkdir -p /opt/isa-l-src \
 ###
 # Avoid out of memory errors in builds
 ###
-ENV MAVEN_OPTS -Xms256m -Xmx1536m
+ENV MAVEN_OPTS -Xms256m -Xmx3072m
 
 # Skip gpg verification when downloading Yetus via yetus-wrapper
 ENV HADOOP_SKIP_YETUS_VERIFICATION true

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -97,12 +98,14 @@ public class TestPersistBlocks {
     conf.setInt(
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
+    conf.setBoolean(HdfsClientConfigKeys.Retry.POLICY_ENABLED_KEY, true);
     MiniDFSCluster cluster = null;
 
     long len = 0;
     FSDataOutputStream stream;
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+      cluster.waitActive();
       FileSystem fs = cluster.getFileSystem();
       // Creating a file with 4096 blockSize to write multiple blocks
       stream = fs.create(FILE_PATH, true, BLOCK_SIZE, (short) 1, BLOCK_SIZE);

+ 29 - 23
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -91,7 +91,6 @@ import org.slf4j.event.Level;
 public class TestDirectoryScanner {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestDirectoryScanner.class);
-  private static final Configuration CONF = new HdfsConfiguration();
   private static final int DEFAULT_GEN_STAMP = 9999;
 
   private MiniDFSCluster cluster;
@@ -103,12 +102,14 @@ public class TestDirectoryScanner {
   private final Random r = new Random();
   private static final int BLOCK_LENGTH = 100;
 
-  static {
-    CONF.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_LENGTH);
-    CONF.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
-    CONF.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
-    CONF.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
+  public Configuration getConfiguration() {
+    Configuration configuration = new HdfsConfiguration();
+    configuration.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_LENGTH);
+    configuration.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, 1);
+    configuration.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+    configuration.setLong(DFSConfigKeys.DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
         getMemlockLimit(Long.MAX_VALUE));
+    return configuration;
   }
 
   @Before
@@ -361,7 +362,8 @@ public class TestDirectoryScanner {
 
   @Test(timeout = 300000)
   public void testRetainBlockOnPersistentStorage() throws Exception {
-    cluster = new MiniDFSCluster.Builder(CONF)
+    Configuration conf = getConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
         .storageTypes(
             new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT })
         .numDataNodes(1).build();
@@ -370,7 +372,7 @@ public class TestDirectoryScanner {
       bpid = cluster.getNamesystem().getBlockPoolId();
       fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
       client = cluster.getFileSystem().getClient();
-      scanner = new DirectoryScanner(fds, CONF);
+      scanner = new DirectoryScanner(fds, conf);
       scanner.setRetainDiffs(true);
       FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
 
@@ -413,8 +415,9 @@ public class TestDirectoryScanner {
         new WriterAppender(new SimpleLayout(), loggerStream);
     rootLogger.addAppender(writerAppender);
 
+    Configuration conf = getConfiguration();
     cluster = new MiniDFSCluster
-        .Builder(CONF)
+        .Builder(conf)
         .storageTypes(new StorageType[] {
             StorageType.RAM_DISK, StorageType.DEFAULT })
         .numDataNodes(1)
@@ -424,7 +427,7 @@ public class TestDirectoryScanner {
       bpid = cluster.getNamesystem().getBlockPoolId();
       fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
       client = cluster.getFileSystem().getClient();
-      scanner = new DirectoryScanner(fds, CONF);
+      scanner = new DirectoryScanner(fds, conf);
       scanner.setRetainDiffs(true);
       FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
 
@@ -464,7 +467,8 @@ public class TestDirectoryScanner {
 
   @Test(timeout = 300000)
   public void testDeleteBlockOnTransientStorage() throws Exception {
-    cluster = new MiniDFSCluster.Builder(CONF)
+    Configuration conf = getConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf)
         .storageTypes(
             new StorageType[] { StorageType.RAM_DISK, StorageType.DEFAULT })
         .numDataNodes(1).build();
@@ -473,7 +477,7 @@ public class TestDirectoryScanner {
       bpid = cluster.getNamesystem().getBlockPoolId();
       fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
       client = cluster.getFileSystem().getClient();
-      scanner = new DirectoryScanner(fds, CONF);
+      scanner = new DirectoryScanner(fds, conf);
       scanner.setRetainDiffs(true);
       FsDatasetTestUtil.stopLazyWriter(cluster.getDataNodes().get(0));
 
@@ -512,16 +516,17 @@ public class TestDirectoryScanner {
   }
 
   public void runTest(int parallelism) throws Exception {
-    cluster = new MiniDFSCluster.Builder(CONF).build();
+    Configuration conf = getConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf).build();
     try {
       cluster.waitActive();
       bpid = cluster.getNamesystem().getBlockPoolId();
       fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
       client = cluster.getFileSystem().getClient();
-      CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY,
           parallelism);
 
-      scanner = new DirectoryScanner(fds, CONF);
+      scanner = new DirectoryScanner(fds, conf);
       scanner.setRetainDiffs(true);
 
       // Add files with 100 blocks
@@ -672,9 +677,9 @@ public class TestDirectoryScanner {
    *
    * @throws Exception thrown on unexpected failure
    */
-  @Test(timeout = 600000)
+  @Test
   public void testThrottling() throws Exception {
-    Configuration conf = new Configuration(CONF);
+    Configuration conf = new Configuration(getConfiguration());
 
     // We need lots of blocks so the report compiler threads have enough to
     // keep them busy while we watch them.
@@ -714,7 +719,7 @@ public class TestDirectoryScanner {
       // Waiting should be about 9x running.
       LOG.info("RATIO: " + ratio);
       assertTrue("Throttle is too restrictive", ratio <= 10f);
-      assertTrue("Throttle is too permissive", ratio >= 7f);
+      assertTrue("Throttle is too permissive" + ratio, ratio >= 7f);
 
       // Test with a different limit
       conf.setInt(
@@ -754,7 +759,7 @@ public class TestDirectoryScanner {
       assertTrue("Throttle is too permissive", ratio >= 7f);
 
       // Test with no limit
-      scanner = new DirectoryScanner(fds, CONF);
+      scanner = new DirectoryScanner(fds, getConfiguration());
       scanner.setRetainDiffs(true);
       scan(blocks, 0, 0, 0, 0, 0);
       scanner.shutdown();
@@ -1095,13 +1100,14 @@ public class TestDirectoryScanner {
    */
   @Test(timeout = 60000)
   public void testExceptionHandlingWhileDirectoryScan() throws Exception {
-    cluster = new MiniDFSCluster.Builder(CONF).build();
+    Configuration conf = getConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf).build();
     try {
       cluster.waitActive();
       bpid = cluster.getNamesystem().getBlockPoolId();
       fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
       client = cluster.getFileSystem().getClient();
-      CONF.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
 
       // Add files with 2 blocks
       createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 2, false);
@@ -1121,7 +1127,7 @@ public class TestDirectoryScanner {
       FsDatasetSpi<? extends FsVolumeSpi> spyFds = Mockito.spy(fds);
       Mockito.doReturn(volReferences).when(spyFds).getFsVolumeReferences();
 
-      scanner = new DirectoryScanner(spyFds, CONF);
+      scanner = new DirectoryScanner(spyFds, conf);
       scanner.setRetainDiffs(true);
       scanner.reconcile();
     } finally {
@@ -1135,7 +1141,7 @@ public class TestDirectoryScanner {
 
   @Test
   public void testDirectoryScannerInFederatedCluster() throws Exception {
-    HdfsConfiguration conf = new HdfsConfiguration(CONF);
+    HdfsConfiguration conf = new HdfsConfiguration(getConfiguration());
     // Create Federated cluster with two nameservices and one DN
     try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java

@@ -324,7 +324,7 @@ public class TestPipelinesFailover {
    * DN running the recovery should then fail to commit the synchronization
    * and a later retry will succeed.
    */
-  @Test(timeout=30000)
+  @Test(timeout=60000)
   public void testFailoverRightBeforeCommitSynchronization() throws Exception {
     final Configuration conf = new Configuration();
     // Disable permissions so that another user can recover the lease.

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestNestedSnapshots.java

@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
-import static org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature.SNAPSHOT_QUOTA_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_MAX_LIMIT;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -61,6 +61,7 @@ public class TestNestedSnapshots {
 
   private static final short REPLICATION = 3;
   private static final long BLOCKSIZE = 1024;
+  private static final int SNAPSHOTLIMIT = 100;
   
   private static final Configuration conf = new Configuration();
   private static MiniDFSCluster cluster;
@@ -68,6 +69,7 @@ public class TestNestedSnapshots {
   
   @Before
   public void setUp() throws Exception {
+    conf.setInt(DFS_NAMENODE_SNAPSHOT_MAX_LIMIT, SNAPSHOTLIMIT);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
         .build();
     cluster.waitActive();
@@ -199,7 +201,7 @@ public class TestNestedSnapshots {
    * Test the snapshot limit of a single snapshottable directory.
    * @throws Exception
    */
-  @Test (timeout=300000)
+  @Test (timeout=600000)
   public void testSnapshotLimit() throws Exception {
     final int step = 1000;
     final String dirStr = "/testSnapshotLimit/dir";
@@ -208,7 +210,8 @@ public class TestNestedSnapshots {
     hdfs.allowSnapshot(dir);
 
     int s = 0;
-    for(; s < SNAPSHOT_QUOTA_DEFAULT; s++) {
+    for(; s < SNAPSHOTLIMIT; s++) {
+      SnapshotTestHelper.LOG.info("Creating snapshot number: {}", s);
       final String snapshotName = "s" + s;
       hdfs.createSnapshot(dir, snapshotName);
 
@@ -226,10 +229,10 @@ public class TestNestedSnapshots {
       SnapshotTestHelper.LOG.info("The exception is expected.", ioe);
     }
 
-    for(int f = 0; f < SNAPSHOT_QUOTA_DEFAULT; f += step) {
+    for(int f = 0; f < SNAPSHOTLIMIT; f += step) {
       final String file = "f" + f;
       s = RANDOM.nextInt(step);
-      for(; s < SNAPSHOT_QUOTA_DEFAULT; s += RANDOM.nextInt(step)) {
+      for(; s < SNAPSHOTLIMIT; s += RANDOM.nextInt(step)) {
         final Path p = SnapshotTestHelper.getSnapshotPath(dir, "s" + s, file);
         //the file #f exists in snapshot #s iff s > f.
         Assert.assertEquals(s > f, hdfs.exists(p));