Преглед изворни кода

HADOOP-18508. S3A: Support parallel integration test runs on same bucket (#5081)

It is now possible to provide a job ID in the maven "job.id" property
hadoop-aws test runs to isolate paths under a the test bucket
under which all tests will be executed.

This will allow independent builds *in different source trees*
to test against the same bucket in parallel, and is designed for
CI testing.

Example:

mvn verify -Dparallel-tests -Droot.tests.enabled=false -Djob.id=1
mvn verify -Droot.tests.enabled=false -Djob.id=2

- Root tests must be be disabled to stop them cleaning up
  the test paths of other test runs.
- Do still regularly run the root tests just to force cleanup
  of the output of any interrupted test suites.  

Contributed by Steve Loughran
Steve Loughran пре 10 месеци
родитељ
комит
2d5fa9e016
18 измењених фајлова са 197 додато и 170 уклоњено
  1. 4 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java
  2. 7 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java
  3. 1 25
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFSMainOperationsLocalFileSystem.java
  4. 0 9
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestFSMainOperationsLocalFileSystem.java
  5. 15 12
      hadoop-tools/hadoop-aws/pom.xml
  6. 46 3
      hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md
  7. 8 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java
  8. 4 2
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java
  9. 21 71
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java
  10. 12 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
  11. 11 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
  12. 15 7
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java
  13. 25 5
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextMainOperations.java
  14. 10 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java
  15. 2 3
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java
  16. 1 1
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java
  17. 2 0
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerToolRootOperations.java
  18. 13 28
      hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3A.java

+ 4 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FSMainOperationsBaseTest.java

@@ -102,7 +102,9 @@ public abstract class FSMainOperationsBaseTest extends FileSystemTestHelper {
   
   @After
   public void tearDown() throws Exception {
-    fSys.delete(new Path(getAbsoluteTestRootPath(fSys), new Path("test")), true);
+    if (fSys != null) {
+      fSys.delete(new Path(getAbsoluteTestRootPath(fSys), new Path("test")), true);
+    }
   }
   
   
@@ -192,7 +194,7 @@ public abstract class FSMainOperationsBaseTest extends FileSystemTestHelper {
   
   @Test
   public void testWDAbsolute() throws IOException {
-    Path absoluteDir = new Path(fSys.getUri() + "/test/existingDir");
+    Path absoluteDir = getTestRootPath(fSys, "test/existingDir");
     fSys.mkdirs(absoluteDir);
     fSys.setWorkingDirectory(absoluteDir);
     Assert.assertEquals(absoluteDir, fSys.getWorkingDirectory());

+ 7 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java

@@ -81,6 +81,12 @@ public abstract class FileContextMainOperationsBaseTest  {
   protected final FileContextTestHelper fileContextTestHelper =
     createFileContextHelper();
 
+  /**
+   * Create the test helper.
+   * Important: this is invoked during the construction of the base class,
+   * so is very brittle.
+   * @return a test helper.
+   */
   protected FileContextTestHelper createFileContextHelper() {
     return new FileContextTestHelper();
   }
@@ -107,7 +113,7 @@ public abstract class FileContextMainOperationsBaseTest  {
   
   private static final byte[] data = getFileData(numBlocks,
       getDefaultBlockSize());
-  
+
   @Before
   public void setUp() throws Exception {
     File testBuildData = GenericTestUtils.getRandomizedTestDir();

+ 1 - 25
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFSMainOperationsLocalFileSystem.java

@@ -21,10 +21,6 @@ package org.apache.hadoop.fs;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
 
 public class TestFSMainOperationsLocalFileSystem extends FSMainOperationsBaseTest {
 
@@ -32,12 +28,6 @@ public class TestFSMainOperationsLocalFileSystem extends FSMainOperationsBaseTes
   protected FileSystem createFileSystem() throws IOException {
     return FileSystem.getLocal(new Configuration());
   }
-    
-  @Override
-  @Before
-  public void setUp() throws Exception {
-    super.setUp();
-  }
   
   static Path wd = null;
   @Override
@@ -46,19 +36,5 @@ public class TestFSMainOperationsLocalFileSystem extends FSMainOperationsBaseTes
       wd = FileSystem.getLocal(new Configuration()).getWorkingDirectory();
     return wd;
   }
-  
-  @Override
-  @After
-  public void tearDown() throws Exception {
-    super.tearDown();
-  }
-  
-  @Test
-  @Override
-  public void testWDAbsolute() throws IOException {
-    Path absoluteDir = getTestRootPath(fSys, "test/existingDir");
-    fSys.mkdirs(absoluteDir);
-    fSys.setWorkingDirectory(absoluteDir);
-    Assert.assertEquals(absoluteDir, fSys.getWorkingDirectory());
-  }
+
 }

+ 0 - 9
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestFSMainOperationsLocalFileSystem.java

@@ -53,14 +53,5 @@ public class TestFSMainOperationsLocalFileSystem extends FSMainOperationsBaseTes
     super.tearDown();
     ViewFileSystemTestSetup.tearDown(this, fcTarget);
   }
-  
-  @Test
-  @Override
-  public void testWDAbsolute() throws IOException {
-    Path absoluteDir = getTestRootPath(fSys, "test/existingDir");
-    fSys.mkdirs(absoluteDir);
-    fSys.setWorkingDirectory(absoluteDir);
-    Assert.assertEquals(absoluteDir, fSys.getWorkingDirectory());
 
-  }
 }

+ 15 - 12
hadoop-tools/hadoop-aws/pom.xml

@@ -56,6 +56,11 @@
 
     <!-- Is prefetch enabled? -->
     <fs.s3a.prefetch.enabled>unset</fs.s3a.prefetch.enabled>
+    <!-- Job ID; allows for parallel jobs on same bucket -->
+    <!-- job.id is used to build the path for tests; default is 00.-->
+    <job.id>00</job.id>
+    <!-- are root tests enabled. Set to false when running parallel jobs on same bucket -->
+    <root.tests.enabled>unset</root.tests.enabled>
   </properties>
 
   <profiles>
@@ -115,14 +120,8 @@
                 <test.build.data>${test.build.data}/${surefire.forkNumber}</test.build.data>
                 <test.build.dir>${test.build.dir}/${surefire.forkNumber}</test.build.dir>
                 <hadoop.tmp.dir>${hadoop.tmp.dir}/${surefire.forkNumber}</hadoop.tmp.dir>
+                <test.unique.fork.id>job-${job.id}-fork-000${surefire.forkNumber}</test.unique.fork.id>
 
-                <!-- Due to a Maven quirk, setting this to just -->
-                <!-- surefire.forkNumber won't do the parameter -->
-                <!-- substitution.  Putting a prefix in front of it like -->
-                <!-- "fork-" makes it work. -->
-                <!-- Important: Those leading 0s are needed to guarantee that -->
-                <!-- trailing three chars are always numeric and unique -->
-                <test.unique.fork.id>fork-000${surefire.forkNumber}</test.unique.fork.id>
                 <!-- Propagate scale parameters -->
                 <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
                 <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
@@ -163,7 +162,7 @@
                     <!-- surefire.forkNumber won't do the parameter -->
                     <!-- substitution.  Putting a prefix in front of it like -->
                     <!-- "fork-" makes it work. -->
-                    <test.unique.fork.id>fork-000${surefire.forkNumber}</test.unique.fork.id>
+                    <test.unique.fork.id>job-${job.id}-fork-000${surefire.forkNumber}</test.unique.fork.id>
                     <!-- Propagate scale parameters -->
                     <fs.s3a.scale.test.enabled>${fs.s3a.scale.test.enabled}</fs.s3a.scale.test.enabled>
                     <fs.s3a.scale.test.huge.filesize>${fs.s3a.scale.test.huge.filesize}</fs.s3a.scale.test.huge.filesize>
@@ -174,14 +173,14 @@
                     <test.default.timeout>${test.integration.timeout}</test.default.timeout>
                     <!-- Prefetch -->
                     <fs.s3a.prefetch.enabled>${fs.s3a.prefetch.enabled}</fs.s3a.prefetch.enabled>
+                    <!-- are root tests enabled. Set to false when running parallel jobs on same bucket -->
+                    <fs.s3a.root.tests.enabled>${root.tests.enabled}</fs.s3a.root.tests.enabled>
+
                   </systemPropertyVariables>
                   <!-- Some tests cannot run in parallel.  Tests that cover -->
                   <!-- access to the root directory must run in isolation -->
                   <!-- from anything else that could modify the bucket. -->
-                  <!-- S3A tests that cover multi-part upload must run in -->
-                  <!-- isolation, because the file system is configured to -->
-                  <!-- purge existing multi-part upload data on -->
-                  <!-- initialization.  MiniYARNCluster has not yet been -->
+                  <!-- MiniYARNCluster has not yet been -->
                   <!-- changed to handle parallel test execution gracefully. -->
                   <!-- Exclude all of these tests from parallel execution, -->
                   <!-- and instead run them sequentially in a separate -->
@@ -228,6 +227,9 @@
                     <fs.s3a.directory.marker.audit>${fs.s3a.directory.marker.audit}</fs.s3a.directory.marker.audit>
                     <!-- Prefetch -->
                     <fs.s3a.prefetch.enabled>${fs.s3a.prefetch.enabled}</fs.s3a.prefetch.enabled>
+                    <!-- are root tests enabled. Set to false when running parallel jobs on same bucket -->
+                    <fs.s3a.root.tests.enabled>${root.tests.enabled}</fs.s3a.root.tests.enabled>
+                    <test.unique.fork.id>job-${job.id}</test.unique.fork.id>
                   </systemPropertyVariables>
                   <!-- Do a sequential run for tests that cannot handle -->
                   <!-- parallel execution. -->
@@ -289,6 +291,7 @@
                     <fs.s3a.directory.marker.audit>${fs.s3a.directory.marker.audit}</fs.s3a.directory.marker.audit>
                     <!-- Prefetch -->
                     <fs.s3a.prefetch.enabled>${fs.s3a.prefetch.enabled}</fs.s3a.prefetch.enabled>
+                    <test.unique.fork.id>job-${job.id}</test.unique.fork.id>
                   </systemPropertyVariables>
                   <forkedProcessTimeoutInSeconds>${fs.s3a.scale.test.timeout}</forkedProcessTimeoutInSeconds>
                 </configuration>

+ 46 - 3
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md

@@ -43,7 +43,7 @@ is a more specific lie and harder to make. And, if you get caught out: you
 lose all credibility with the project.
 
 You don't need to test from a VM within the AWS infrastructure; with the
-`-Dparallel=tests` option the non-scale tests complete in under ten minutes.
+`-Dparallel-tests` option the non-scale tests complete in under twenty minutes.
 Because the tests clean up after themselves, they are also designed to be low
 cost. It's neither hard nor expensive to run the tests; if you can't,
 there's no guarantee your patch works. The reviewers have enough to do, and
@@ -539,12 +539,51 @@ Otherwise, set a large timeout in `fs.s3a.scale.test.timeout`
 The tests are executed in an order to only clean up created files after
 the end of all the tests. If the tests are interrupted, the test data will remain.
 
+## <a name="CI"/> Testing through continuous integration
+
+### Parallel CI builds.
+For CI testing of the module, including the integration tests,
+it is generally necessary to support testing multiple PRs simultaneously.
+
+To do this
+1. A job ID must be supplied in the `job.id` property, so each job works on an isolated directory
+   tree. This should be a number or unique string, which will be used within a path element, so
+   must only contain characters valid in an S3/hadoop path element.
+2. Root directory tests need to be disabled by setting `fs.s3a.root.tests.enabled` to
+   `false`, either in the command line to maven or in the XML configurations.
+
+```
+mvn verify -T 1C -Dparallel-tests -DtestsThreadCount=14 -Dscale -Dfs.s3a.root.tests.enabled=false -Djob.id=001
+```
+
+This parallel execution feature is only for isolated builds sharing a single S3 bucket; it does
+not support parallel builds and tests from the same local source tree.
+
+Without the root tests being executed, set up a scheduled job to purge the test bucket of all
+data on a regular basis, to keep costs down.
+The easiest way to do this is to have a bucket lifecycle rule for the bucket to delete all files more than a few days old,
+alongside one to abort all pending uploads more than 24h old.
+
+
+### Securing CI builds
+
+It's clearly unsafe to have CI infrastructure testing PRs submitted to apache github account
+with AWS credentials -which is why it isn't done by the Yetus-initiated builds.
+
+Anyone doing this privately should:
+* Review incoming patches before triggering the tests.
+* Have a dedicated IAM role with restricted access to the test bucket, any KMS keys used, and the
+  external bucket containing the CSV test file.
+* Have a build process which generates short-lived session credentials for this role.
+* Run the tests in an EC2 VM/container which collects the restricted IAM credentials
+  from the IAM instance/container credentials provider.
+
 ## <a name="load"></a> Load tests.
 
-Some are designed to overload AWS services with more
+Some tests are designed to overload AWS services with more
 requests per second than an AWS account is permitted.
 
-The operation of these test maybe observable to other users of the same
+The operation of these tests may be observable to other users of the same
 account -especially if they are working in the AWS region to which the
 tests are targeted.
 
@@ -556,6 +595,10 @@ They do not run automatically: they must be explicitly run from the command line
 
 Look in the source for these and reads the Javadocs before executing.
 
+Note: one fear here was that asking for two many session/role credentials in a short period
+of time would actually lock an account out of a region. It doesn't: it simply triggers
+throttling of STS requests.
+
 ## <a name="alternate_s3"></a> Testing against non-AWS S3 Stores.
 
 The S3A filesystem is designed to work with S3 stores which implement

+ 8 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractRootDir.java

@@ -27,6 +27,8 @@ import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeSkipRootTests;
+
 /**
  * root dir operations against an S3 bucket.
  */
@@ -36,6 +38,12 @@ public class ITestS3AContractRootDir extends
   private static final Logger LOG =
       LoggerFactory.getLogger(ITestS3AContractRootDir.class);
 
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    maybeSkipRootTests(getFileSystem().getConf());
+  }
+
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);

+ 4 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AConfiguration.java

@@ -386,8 +386,10 @@ public class ITestS3AConfiguration {
           s3Configuration.pathStyleAccessEnabled());
       byte[] file = ContractTestUtils.toAsciiByteArray("test file");
       ContractTestUtils.writeAndRead(fs,
-          new Path("/path/style/access/testFile"), file, file.length,
-              (int) conf.getLongBytes(Constants.FS_S3A_BLOCK_SIZE, file.length), false, true);
+          createTestPath(new Path("/path/style/access/testFile")),
+          file, file.length,
+          (int) conf.getLongBytes(Constants.FS_S3A_BLOCK_SIZE, file.length),
+          false, true);
     } catch (final AWSRedirectException e) {
       LOG.error("Caught exception: ", e);
       // Catch/pass standard path style access behaviour when live bucket

+ 21 - 71
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEncryptionSSEC.java

@@ -20,13 +20,9 @@ package org.apache.hadoop.fs.s3a;
 
 import java.io.IOException;
 import java.nio.file.AccessDeniedException;
-import java.util.Arrays;
-import java.util.Collection;
 
 import org.assertj.core.api.Assertions;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,19 +33,14 @@ import org.apache.hadoop.io.IOUtils;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
-import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY;
-import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_DELETE;
-import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_KEEP;
 import static org.apache.hadoop.fs.s3a.Constants.ETAG_CHECKSUM_ENABLED;
 import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_ALGORITHM;
 import static org.apache.hadoop.fs.s3a.Constants.S3_ENCRYPTION_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_ALGORITHM;
 import static org.apache.hadoop.fs.s3a.Constants.SERVER_SIDE_ENCRYPTION_KEY;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.createTestPath;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.getTestBucketName;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeSkipRootTests;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfEncryptionTestsDisabled;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 /**
@@ -60,7 +51,6 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
  * Equally "vexing" has been the optimizations of getFileStatus(), wherein
  * LIST comes before HEAD path + /
  */
-@RunWith(Parameterized.class)
 public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
 
   private static final String SERVICE_AMAZON_S3_STATUS_CODE_403
@@ -75,31 +65,11 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
       = "msdo3VvvZznp66Gth58a91Hxe/UpExMkwU9BHkIjfW8=";
   private static final int TEST_FILE_LEN = 2048;
 
-  /**
-   * Parameterization.
-   */
-  @Parameterized.Parameters(name = "{0}")
-  public static Collection<Object[]> params() {
-    return Arrays.asList(new Object[][]{
-        {"keep-markers", true},
-        {"delete-markers", false}
-    });
-  }
-
-  /**
-   * Parameter: should directory markers be retained?
-   */
-  private final boolean keepMarkers;
-
   /**
    * Filesystem created with a different key.
    */
   private S3AFileSystem fsKeyB;
 
-  public ITestS3AEncryptionSSEC(final String name,
-      final boolean keepMarkers) {
-    this.keepMarkers = keepMarkers;
-  }
 
   @SuppressWarnings("deprecation")
   @Override
@@ -108,16 +78,11 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     String bucketName = getTestBucketName(conf);
     // directory marker options
     removeBaseAndBucketOverrides(bucketName, conf,
-        DIRECTORY_MARKER_POLICY,
         ETAG_CHECKSUM_ENABLED,
         S3_ENCRYPTION_ALGORITHM,
         S3_ENCRYPTION_KEY,
         SERVER_SIDE_ENCRYPTION_ALGORITHM,
         SERVER_SIDE_ENCRYPTION_KEY);
-    conf.set(DIRECTORY_MARKER_POLICY,
-        keepMarkers
-            ? DIRECTORY_MARKER_POLICY_KEEP
-            : DIRECTORY_MARKER_POLICY_DELETE);
     conf.set(S3_ENCRYPTION_ALGORITHM,
         getSSEAlgorithm().getMethod());
     conf.set(S3_ENCRYPTION_KEY, KEY_1);
@@ -129,6 +94,9 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   public void setup() throws Exception {
     super.setup();
     assumeEnabled();
+    // although not a root dir test, this confuses paths enough it shouldn't be run in
+    // parallel with other jobs
+    maybeSkipRootTests(getConfiguration());
   }
 
   @Override
@@ -154,7 +122,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
         () -> {
           int len = TEST_FILE_LEN;
           describe("Create an encrypted file of size " + len);
-          Path src = path("testCreateFileAndReadWithDifferentEncryptionKey");
+          Path src = methodPath();
           writeThenReadFile(src, len);
 
           //extract the test FS
@@ -174,7 +142,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
   @Test
   public void testCreateSubdirWithDifferentKey() throws Exception {
-    Path base = path("testCreateSubdirWithDifferentKey");
+    Path base = methodPath();
     Path nestedDirectory = new Path(base, "nestedDir");
     fsKeyB = createNewFileSystemWithSSECKey(
         KEY_2);
@@ -213,9 +181,10 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
   @Test
   public void testRenameFile() throws Exception {
-    Path src = path("original-path.txt");
+    final Path base = methodPath();
+    Path src = new Path(base, "original-path.txt");
     writeThenReadFile(src, TEST_FILE_LEN);
-    Path newPath = path("different-path.txt");
+    Path newPath = new Path(base, "different-path.txt");
     getFileSystem().rename(src, newPath);
     byte[] data = dataset(TEST_FILE_LEN, 'a', 'z');
     ContractTestUtils.verifyFileContents(getFileSystem(), newPath, data);
@@ -228,11 +197,11 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   @Test
   public void testListEncryptedDir() throws Exception {
 
-    Path pathABC = path("testListEncryptedDir/a/b/c/");
+    Path pathABC = new Path(methodPath(), "a/b/c/");
     Path pathAB = pathABC.getParent();
     Path pathA = pathAB.getParent();
 
-    Path nestedDirectory = createTestPath(pathABC);
+    Path nestedDirectory = pathABC;
     assertTrue(getFileSystem().mkdirs(nestedDirectory));
 
     fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
@@ -261,7 +230,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
   @Test
   public void testListStatusEncryptedDir() throws Exception {
 
-    Path pathABC = path("testListStatusEncryptedDir/a/b/c/");
+    Path pathABC = new Path(methodPath(), "a/b/c/");
     Path pathAB = pathABC.getParent();
     Path pathA = pathAB.getParent();
     assertTrue(getFileSystem().mkdirs(pathABC));
@@ -296,7 +265,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
   @Test
   public void testListStatusEncryptedFile() throws Exception {
-    Path pathABC = path("testListStatusEncryptedFile/a/b/c/");
+    Path pathABC = new Path(methodPath(), "a/b/c/");
     assertTrue("mkdirs failed", getFileSystem().mkdirs(pathABC));
 
     Path fileToStat = new Path(pathABC, "fileToStat.txt");
@@ -305,23 +274,9 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
     fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
 
     //Until this point, no exception is thrown about access
-    if (statusProbesCheckS3(fsKeyB, fileToStat)) {
-      intercept(AccessDeniedException.class,
-          SERVICE_AMAZON_S3_STATUS_CODE_403,
-          () -> fsKeyB.listStatus(fileToStat));
-    } else {
-      fsKeyB.listStatus(fileToStat);
-    }
-  }
-
-  /**
-   * Do file status probes check S3?
-   * @param fs filesystem
-   * @param path file path
-   * @return true if check for a path being a file will issue a HEAD request.
-   */
-  private boolean statusProbesCheckS3(S3AFileSystem fs, Path path) {
-    return true;
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        () -> fsKeyB.listStatus(fileToStat));
   }
 
   /**
@@ -332,22 +287,17 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
   @Test
   public void testDeleteEncryptedObjectWithDifferentKey() throws Exception {
-    //requireUnguardedFilesystem();
-    Path pathABC = path("testDeleteEncryptedObjectWithDifferentKey/a/b/c/");
 
+    Path pathABC = new Path(methodPath(), "a/b/c/");
     Path pathAB = pathABC.getParent();
     Path pathA = pathAB.getParent();
     assertTrue(getFileSystem().mkdirs(pathABC));
     Path fileToDelete = new Path(pathABC, "filetobedeleted.txt");
     writeThenReadFile(fileToDelete, TEST_FILE_LEN);
     fsKeyB = createNewFileSystemWithSSECKey(KEY_4);
-    if (statusProbesCheckS3(fsKeyB, fileToDelete)) {
-      intercept(AccessDeniedException.class,
-          SERVICE_AMAZON_S3_STATUS_CODE_403,
-          () -> fsKeyB.delete(fileToDelete, false));
-    } else {
-      fsKeyB.delete(fileToDelete, false);
-    }
+    intercept(AccessDeniedException.class,
+        SERVICE_AMAZON_S3_STATUS_CODE_403,
+        () -> fsKeyB.delete(fileToDelete, false));
     //This is possible
     fsKeyB.delete(pathABC, true);
     fsKeyB.delete(pathAB, true);
@@ -360,7 +310,7 @@ public class ITestS3AEncryptionSSEC extends AbstractTestS3AEncryption {
    */
   @Test
   public void testChecksumRequiresReadAccess() throws Throwable {
-    Path path = path("tagged-file");
+    Path path = methodPath();
     S3AFileSystem fs = getFileSystem();
     touch(fs, path);
     Assertions.assertThat(fs.getFileChecksum(path))

+ 12 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java

@@ -193,6 +193,8 @@ public interface S3ATestConstants {
 
   /**
    * Fork ID passed down from maven if the test is running in parallel.
+   * If a build was also executed with job.id set, this is included in
+   * the fork ID.
    */
   String TEST_UNIQUE_FORK_ID = "test.unique.fork.id";
   String TEST_STS_ENABLED = "test.fs.s3a.sts.enabled";
@@ -273,4 +275,14 @@ public interface S3ATestConstants {
    * AWS ireland region.
    */
   String EU_WEST_1 = "eu-west-1";
+
+  /**
+   * System property for root tests being enabled: {@value}.
+   */
+  String ROOT_TESTS_ENABLED = "fs.s3a.root.tests.enabled";
+
+  /**
+   * Default policy on root tests: {@value}.
+   */
+  boolean DEFAULT_ROOT_TESTS_ENABLED = true;
 }

+ 11 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java

@@ -601,12 +601,13 @@ public final class S3ATestUtils {
   /**
    * Create a test path, using the value of
    * {@link S3ATestConstants#TEST_UNIQUE_FORK_ID} if it is set.
+   * This path is *not* qualified.
    * @param defVal default value
    * @return a path
    */
   public static Path createTestPath(Path defVal) {
     String testUniqueForkId =
-        System.getProperty(S3ATestConstants.TEST_UNIQUE_FORK_ID);
+        System.getProperty(TEST_UNIQUE_FORK_ID);
     return testUniqueForkId == null ? defVal :
         new Path("/" + testUniqueForkId, "test");
   }
@@ -1738,6 +1739,15 @@ public final class S3ATestUtils {
     removeBaseAndBucketOverrides(conf, PREFETCH_ENABLED_KEY);
   }
 
+  /**
+   * Skip root tests if the system properties/config says so.
+   * @param conf configuration to check
+   */
+  public static void maybeSkipRootTests(Configuration conf) {
+    assume("Root tests disabled",
+        getTestPropertyBool(conf, ROOT_TESTS_ENABLED, DEFAULT_ROOT_TESTS_ENABLED));
+  }
+
   /**
    * Does this FS support multi object delete?
    * @param fs filesystem

+ 15 - 7
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a.commit.terasort;
 
 import java.io.File;
 import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.Collection;
@@ -43,6 +44,7 @@ import org.apache.hadoop.examples.terasort.TeraSort;
 import org.apache.hadoop.examples.terasort.TeraSortConfigKeys;
 import org.apache.hadoop.examples.terasort.TeraValidate;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.fs.s3a.commit.AbstractYarnClusterITest;
 import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter;
@@ -118,7 +120,7 @@ public class ITestTerasortOnS3A extends AbstractYarnClusterITest {
    *
    * @return the committer binding for this run.
    */
-  @Parameterized.Parameters(name = "{0}")
+  @Parameterized.Parameters(name = "{0}-memory={1}")
   public static Collection<Object[]> params() {
     return Arrays.asList(new Object[][]{
         {DirectoryStagingCommitter.NAME, false},
@@ -143,6 +145,11 @@ public class ITestTerasortOnS3A extends AbstractYarnClusterITest {
     prepareToTerasort();
   }
 
+  @Override
+  protected void deleteTestDirInTeardown() throws IOException {
+    /* no-op */
+  }
+
   /**
    * Set up the job conf with the options for terasort chosen by the scale
    * options.
@@ -180,14 +187,14 @@ public class ITestTerasortOnS3A extends AbstractYarnClusterITest {
    * The paths used must be unique across parameterized runs but
    * common across all test cases in a single parameterized run.
    */
-  private void prepareToTerasort() {
+  private void prepareToTerasort() throws IOException {
     // small sample size for faster runs
-    terasortPath = new Path("/terasort-" + committerName + "-" + trackCommitsInMemory)
-        .makeQualified(getFileSystem());
+    terasortPath = getFileSystem().qualify(
+        new Path(S3ATestUtils.createTestPath(new Path("terasort-test")),
+            "terasort-" + committerName + "-" + trackCommitsInMemory));
     sortInput = new Path(terasortPath, "sortin");
     sortOutput = new Path(terasortPath, "sortout");
     sortValidate = new Path(terasortPath, "validate");
-
   }
 
   /**
@@ -254,7 +261,7 @@ public class ITestTerasortOnS3A extends AbstractYarnClusterITest {
    */
   @Test
   public void test_100_terasort_setup() throws Throwable {
-    describe("Setting up for a terasort");
+    describe("Setting up for a terasort with path of %s", terasortPath);
 
     getFileSystem().delete(terasortPath, true);
     completedStages = new HashMap<>();
@@ -339,7 +346,8 @@ public class ITestTerasortOnS3A extends AbstractYarnClusterITest {
     stage.accept("teravalidate");
     stage.accept("overall");
     String text = results.toString();
-    File resultsFile = new File(getReportDir(), committerName + ".csv");
+    File resultsFile = new File(getReportDir(),
+        String.format("%s-%s.csv", committerName, trackCommitsInMemory));
     FileUtils.write(resultsFile, text, StandardCharsets.UTF_8);
     LOG.info("Results are in {}\n{}", resultsFile, text);
   }

+ 25 - 5
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextMainOperations.java

@@ -3,7 +3,7 @@
  * you may not use this file except in compliance with the License.
  * You may obtain a copy of the License at
  *
- *      http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
@@ -11,21 +11,28 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.hadoop.fs.s3a.fileContext;
 
 import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileContextMainOperationsBaseTest;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
+import java.util.UUID;
+
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContextMainOperationsBaseTest;
+import org.apache.hadoop.fs.FileContextTestHelper;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+
 /**
  * S3A implementation of FileContextMainOperationsBaseTest.
  */
 public class ITestS3AFileContextMainOperations
-        extends FileContextMainOperationsBaseTest {
+    extends FileContextMainOperationsBaseTest {
+
 
   @Before
   public void setUp() throws IOException, Exception {
@@ -34,6 +41,19 @@ public class ITestS3AFileContextMainOperations
     super.setUp();
   }
 
+
+  /**
+   * Called before even our own constructor and fields are
+   * inited.
+   * @return a test helper using the s3a test path.
+   */
+  @Override
+  protected FileContextTestHelper createFileContextHelper() {
+    Path testPath =
+        S3ATestUtils.createTestPath(new Path("/" + UUID.randomUUID()));
+    return new FileContextTestHelper(testPath.toUri().toString());
+  }
+
   @Override
   protected boolean listCorruptedBlocksSupported() {
     return false;

+ 10 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java

@@ -113,6 +113,16 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
         DEFAULT_HUGE_FILESIZE);
   }
 
+  /**
+   * Test dir deletion is removed from test case teardown so the
+   * subsequent tests see the output.
+   * @throws IOException failure
+   */
+  @Override
+  protected void deleteTestDirInTeardown() throws IOException {
+    /* no-op */
+  }
+
   /**
    * Get the name of this test suite, which is used in path generation.
    * Base implementation uses {@link #getBlockOutputBufferName()} for this.

+ 2 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AConcurrentOps.java

@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer;
 import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3ATestUtils;
 
 import org.junit.After;
 import org.junit.Test;
@@ -67,8 +66,8 @@ public class ITestS3AConcurrentOps extends S3AScaleTestBase {
     super.setup();
     auxFs = getNormalFileSystem();
 
-    testRoot = path("/ITestS3AConcurrentOps");
-    testRoot = S3ATestUtils.createTestPath(testRoot);
+    // this is set to the method path, even in test setup.
+    testRoot = methodPath();
   }
 
   private S3AFileSystem getNormalFileSystem() throws Exception {

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java

@@ -83,7 +83,7 @@ public class S3AScaleTestBase extends AbstractS3ATestBase {
   @Override
   public void setup() throws Exception {
     super.setup();
-    testPath = path("/tests3ascale");
+    testPath = path("tests3ascale");
     LOG.debug("Scale test operation count = {}", getOperationCount());
     enabled = getTestPropertyBool(
         getConf(),

+ 2 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/tools/ITestMarkerToolRootOperations.java

@@ -26,6 +26,7 @@ import org.junit.runners.MethodSorters;
 
 import org.apache.hadoop.fs.Path;
 
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.maybeSkipRootTests;
 import static org.apache.hadoop.fs.s3a.tools.MarkerTool.AUDIT;
 import static org.apache.hadoop.fs.s3a.tools.MarkerTool.CLEAN;
 import static org.apache.hadoop.fs.s3a.tools.MarkerTool.MARKERS;
@@ -42,6 +43,7 @@ public class ITestMarkerToolRootOperations extends AbstractMarkerToolTest {
   @Override
   public void setup() throws Exception {
     super.setup();
+    maybeSkipRootTests(getConfiguration());
     rootPath = getFileSystem().makeQualified(new Path("/"));
   }
 

+ 13 - 28
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/yarn/ITestS3A.java

@@ -17,49 +17,34 @@
  */
 package org.apache.hadoop.fs.s3a.yarn;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.FsStatus;
-import org.apache.hadoop.fs.Path;
+import java.util.EnumSet;
 
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 
-import java.util.EnumSet;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
 /**
  * S3A tests through the {@link FileContext} API.
  */
-public class ITestS3A {
+public class ITestS3A  extends AbstractS3ATestBase {
   private FileContext fc;
 
   @Rule
   public final Timeout testTimeout = new Timeout(90000);
 
-  @Before
-  public void setUp() throws Exception {
-    Configuration conf = new Configuration();
-    fc = S3ATestUtils.createTestFileContext(conf);
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (fc != null) {
-      fc.delete(getTestPath(), true);
-    }
-  }
 
-  protected Path getTestPath() {
-    return S3ATestUtils.createTestPath(new Path("/tests3afc"));
+  @Override
+  public void setup() throws Exception {
+    super.setup();
+    fc = S3ATestUtils.createTestFileContext(getConfiguration());
   }
 
   @Test
@@ -77,7 +62,7 @@ public class ITestS3A {
 
   @Test
   public void testS3ACreateFileInSubDir() throws Exception {
-    Path dirPath = getTestPath();
+    Path dirPath = methodPath();
     fc.mkdir(dirPath, FileContext.DIR_DEFAULT_PERM, true);
     Path filePath = new Path(dirPath, "file");
     try (FSDataOutputStream file = fc.create(filePath, EnumSet.of(CreateFlag