浏览代码

HADOOP-19072. S3A: expand optimisations on stores with "fs.s3a.performance.flags" for mkdir (#6543)

If the flag list in fs.s3a.performance.flags includes "mkdir"
then the safety check of a walk up the tree to look for a parent directory,
-done to verify a directory isn't being created under a file- are skipped.

This saves the cost of multiple list operations.

Includes:

HADOOP-19072. S3A: Override fs.s3a.performance.flags for tests (ADDENDUM) (#6985)

This is a followup to #6543 which ensures all test pass in configurations where
fs.s3a.performance.flags is set to "*" or contains "mkdirs"

Contributed by VJ Jasani
Viraj Jasani 8 月之前
父节点
当前提交
5ea68e9548

+ 2 - 2
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md

@@ -200,8 +200,8 @@ Prioritize file creation performance over safety checks for filesystem consisten
 This:
 This:
 1. Skips the `LIST` call which makes sure a file is being created over a directory.
 1. Skips the `LIST` call which makes sure a file is being created over a directory.
    Risk: a file is created over a directory.
    Risk: a file is created over a directory.
-1. Ignores the overwrite flag.
-1. Never issues a `DELETE` call to delete parent directory markers.
+2. Ignores the overwrite flag.
+3. Never issues a `DELETE` call to delete parent directory markers.
 
 
 It is possible to probe an S3A Filesystem instance for this capability through
 It is possible to probe an S3A Filesystem instance for this capability through
 the `hasPathCapability(path, "fs.s3a.create.performance")` check.
 the `hasPathCapability(path, "fs.s3a.create.performance")` check.

+ 12 - 9
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextCreateMkdirBaseTest.java

@@ -27,6 +27,7 @@ import org.junit.Test;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsDirectory;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
 import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
 
 
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.slf4j.event.Level;
 import org.slf4j.event.Level;
@@ -55,7 +56,10 @@ public abstract class FileContextCreateMkdirBaseTest {
 
 
   protected final FileContextTestHelper fileContextTestHelper;
   protected final FileContextTestHelper fileContextTestHelper;
   protected static FileContext fc;
   protected static FileContext fc;
-      
+
+  public static final String MKDIR_FILE_PRESENT_ERROR =
+      " should have failed as a file was present";
+
   static {
   static {
     GenericTestUtils.setLogLevel(FileSystem.LOG, Level.DEBUG);
     GenericTestUtils.setLogLevel(FileSystem.LOG, Level.DEBUG);
   }
   }
@@ -128,7 +132,7 @@ public abstract class FileContextCreateMkdirBaseTest {
   }
   }
 
 
   @Test
   @Test
-  public void testMkdirRecursiveWithExistingFile() throws IOException {
+  public void testMkdirRecursiveWithExistingFile() throws Exception {
     Path f = getTestRootPath(fc, "NonExistant3/aDir");
     Path f = getTestRootPath(fc, "NonExistant3/aDir");
     fc.mkdir(f, FileContext.DEFAULT_PERM, true);
     fc.mkdir(f, FileContext.DEFAULT_PERM, true);
     assertIsDirectory(fc.getFileStatus(f));
     assertIsDirectory(fc.getFileStatus(f));
@@ -141,13 +145,12 @@ public abstract class FileContextCreateMkdirBaseTest {
 
 
     // try creating another folder which conflicts with filePath
     // try creating another folder which conflicts with filePath
     Path dirPath = new Path(filePath, "bDir/cDir");
     Path dirPath = new Path(filePath, "bDir/cDir");
-    try {
-      fc.mkdir(dirPath, FileContext.DEFAULT_PERM, true);
-      Assert.fail("Mkdir for " + dirPath
-          + " should have failed as a file was present");
-    } catch(IOException e) {
-      // failed as expected
-    }
+    intercept(
+        IOException.class,
+        null,
+        "Mkdir for " + dirPath + MKDIR_FILE_PRESENT_ERROR,
+        () -> fc.mkdir(dirPath, FileContext.DEFAULT_PERM, true)
+    );
   }
   }
 
 
   @Test
   @Test

+ 5 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractMkdirTest.java

@@ -35,6 +35,9 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
  */
  */
 public abstract class AbstractContractMkdirTest extends AbstractFSContractTestBase {
 public abstract class AbstractContractMkdirTest extends AbstractFSContractTestBase {
 
 
+  public static final String MKDIRS_NOT_FAILED_OVER_FILE =
+      "mkdirs did not fail over a file but returned ";
+
   @Test
   @Test
   public void testMkDirRmDir() throws Throwable {
   public void testMkDirRmDir() throws Throwable {
     FileSystem fs = getFileSystem();
     FileSystem fs = getFileSystem();
@@ -66,7 +69,7 @@ public abstract class AbstractContractMkdirTest extends AbstractFSContractTestBa
     createFile(getFileSystem(), path, false, dataset);
     createFile(getFileSystem(), path, false, dataset);
     try {
     try {
       boolean made = fs.mkdirs(path);
       boolean made = fs.mkdirs(path);
-      fail("mkdirs did not fail over a file but returned " + made
+      fail(MKDIRS_NOT_FAILED_OVER_FILE + made
             + "; " + ls(path));
             + "; " + ls(path));
     } catch (ParentNotDirectoryException | FileAlreadyExistsException e) {
     } catch (ParentNotDirectoryException | FileAlreadyExistsException e) {
       //parent is a directory
       //parent is a directory
@@ -93,7 +96,7 @@ public abstract class AbstractContractMkdirTest extends AbstractFSContractTestBa
     Path child = new Path(path,"child-to-mkdir");
     Path child = new Path(path,"child-to-mkdir");
     try {
     try {
       boolean made = fs.mkdirs(child);
       boolean made = fs.mkdirs(child);
-      fail("mkdirs did not fail over a file but returned " + made
+      fail(MKDIRS_NOT_FAILED_OVER_FILE + made
            + "; " + ls(path));
            + "; " + ls(path));
     } catch (ParentNotDirectoryException | FileAlreadyExistsException e) {
     } catch (ParentNotDirectoryException | FileAlreadyExistsException e) {
       //parent is a directory
       //parent is a directory

+ 5 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -3822,7 +3822,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
             createStoreContext(),
             createStoreContext(),
             path,
             path,
             createMkdirOperationCallbacks(),
             createMkdirOperationCallbacks(),
-            isMagicCommitPath(path)));
+            isMagicCommitPath(path),
+            performanceFlags.enabled(PerformanceFlagEnum.Mkdir)));
   }
   }
 
 
   /**
   /**
@@ -4275,7 +4276,9 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
           new MkdirOperation(
           new MkdirOperation(
               storeContext,
               storeContext,
               path,
               path,
-              createMkdirOperationCallbacks(), false));
+              createMkdirOperationCallbacks(),
+              false,
+              performanceFlags.enabled(PerformanceFlagEnum.Mkdir)));
     }
     }
   }
   }
 
 

+ 61 - 16
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java

@@ -26,6 +26,8 @@ import java.util.Set;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
@@ -54,30 +56,54 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus;
  *   <li>If needed, one PUT</li>
  *   <li>If needed, one PUT</li>
  * </ol>
  * </ol>
  */
  */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
 public class MkdirOperation extends ExecutingStoreOperation<Boolean> {
 public class MkdirOperation extends ExecutingStoreOperation<Boolean> {
 
 
   private static final Logger LOG = LoggerFactory.getLogger(
   private static final Logger LOG = LoggerFactory.getLogger(
       MkdirOperation.class);
       MkdirOperation.class);
 
 
+  /**
+   * Path of the directory to be created.
+   */
   private final Path dir;
   private final Path dir;
 
 
+  /**
+   * Mkdir Callbacks object to be used by the Mkdir operation.
+   */
   private final MkdirCallbacks callbacks;
   private final MkdirCallbacks callbacks;
 
 
   /**
   /**
-   * Should checks for ancestors existing be skipped?
-   * This flag is set when working with magic directories.
+   * Whether to skip the validation of the parent directory.
+   */
+  private final boolean performanceMkdir;
+
+  /**
+   * Whether the path is magic commit path.
    */
    */
   private final boolean isMagicPath;
   private final boolean isMagicPath;
 
 
+  /**
+   * Initialize Mkdir Operation context for S3A.
+   *
+   * @param storeContext Store context.
+   * @param dir Dir path of the directory.
+   * @param callbacks MkdirCallbacks object used by the Mkdir operation.
+   * @param isMagicPath True if the path is magic commit path.
+   * @param performanceMkdir If true, skip validation of the parent directory
+   * structure.
+   */
   public MkdirOperation(
   public MkdirOperation(
       final StoreContext storeContext,
       final StoreContext storeContext,
       final Path dir,
       final Path dir,
       final MkdirCallbacks callbacks,
       final MkdirCallbacks callbacks,
-      final boolean isMagicPath) {
+      final boolean isMagicPath,
+      final boolean performanceMkdir) {
     super(storeContext);
     super(storeContext);
     this.dir = dir;
     this.dir = dir;
     this.callbacks = callbacks;
     this.callbacks = callbacks;
     this.isMagicPath = isMagicPath;
     this.isMagicPath = isMagicPath;
+    this.performanceMkdir = performanceMkdir;
   }
   }
 
 
   /**
   /**
@@ -124,7 +150,32 @@ public class MkdirOperation extends ExecutingStoreOperation<Boolean> {
       return true;
       return true;
     }
     }
 
 
-    // Walk path to root, ensuring closest ancestor is a directory, not file
+    // if performance creation mode is set, no need to check
+    // whether the closest ancestor is dir.
+    if (!performanceMkdir) {
+      verifyFileStatusOfClosestAncestor();
+    }
+
+    // if we get here there is no directory at the destination.
+    // so create one.
+
+    // Create the marker file, delete the parent entries
+    // if the filesystem isn't configured to retain them
+    callbacks.createFakeDirectory(dir, false);
+    return true;
+  }
+
+  /**
+   * Verify the file status of the closest ancestor, if it is
+   * dir, the mkdir operation should proceed. If it is file,
+   * the mkdir operation should throw error.
+   *
+   * @throws IOException If either file status could not be retrieved,
+   * or if the closest ancestor is a file.
+   */
+  private void verifyFileStatusOfClosestAncestor() throws IOException {
+    FileStatus fileStatus;
+    // Walk path to root, ensuring the closest ancestor is a directory, not file
     Path fPart = dir.getParent();
     Path fPart = dir.getParent();
     try {
     try {
       while (fPart != null && !fPart.isRoot()) {
       while (fPart != null && !fPart.isRoot()) {
@@ -140,24 +191,18 @@ public class MkdirOperation extends ExecutingStoreOperation<Boolean> {
         }
         }
 
 
         // there's a file at the parent entry
         // there's a file at the parent entry
-        throw new FileAlreadyExistsException(String.format(
-            "Can't make directory for path '%s' since it is a file.",
-            fPart));
+        throw new FileAlreadyExistsException(
+            String.format(
+                "Can't make directory for path '%s' since it is a file.",
+                fPart));
       }
       }
     } catch (AccessDeniedException e) {
     } catch (AccessDeniedException e) {
       LOG.info("mkdirs({}}: Access denied when looking"
       LOG.info("mkdirs({}}: Access denied when looking"
               + " for parent directory {}; skipping checks",
               + " for parent directory {}; skipping checks",
-          dir, fPart);
+          dir,
+          fPart);
       LOG.debug("{}", e, e);
       LOG.debug("{}", e, e);
     }
     }
-
-    // if we get here there is no directory at the destination.
-    // so create one.
-
-    // Create the marker file, delete the parent entries
-    // if the filesystem isn't configured to retain them
-    callbacks.createFakeDirectory(dir, false);
-    return true;
   }
   }
 
 
   /**
   /**

+ 20 - 1
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md

@@ -299,8 +299,11 @@ understands the risks.
 | *Option* | *Meaning*          | Since |
 | *Option* | *Meaning*          | Since |
 |----------|--------------------|:------|
 |----------|--------------------|:------|
 | `create` | Create Performance | 3.4.1 |
 | `create` | Create Performance | 3.4.1 |
+| `mkdir`  | Mkdir Performance  | 3.4.1 |
 
 
-The `create` flag has the same semantics as [`fs.s3a.create.performance`](#create-performance)
+
+* The `create` flag has the same semantics as [`fs.s3a.create.performance`](#create-performance)
+* The `mkdir` flag semantics are explained in [Mkdir Performance](#mkdir-performance)
 
 
 
 
 ### <a name="create-performance"></a> Create Performance `fs.s3a.create.performance`
 ### <a name="create-performance"></a> Create Performance `fs.s3a.create.performance`
@@ -321,6 +324,22 @@ It may however result in
 
 
 Use with care, and, ideally, enable versioning on the S3 store.
 Use with care, and, ideally, enable versioning on the S3 store.
 
 
+
+### <a name="mkdir-performance"></a> Mkdir Performance
+
+`fs.s3a.performance.flag` flag option `mkdir`:
+
+* Mkdir does not check whether the parent is directory or file.
+
+This avoids the verification of the file status of the parent file
+or the closest ancestor. Unlike the default mkdir operation, if the
+parent is not a directory, the mkdir operation does not throw any
+error.
+
+This option can help with mkdir performance improvement but must be used
+only if the person setting them understands the above-mentioned risk.
+
+
 ### <a name="threads"></a> Thread and connection pool settings.
 ### <a name="threads"></a> Thread and connection pool settings.
 
 
 Each S3A client interacting with a single bucket, as a single user, has its
 Each S3A client interacting with a single bucket, as a single user, has its

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

@@ -22,11 +22,26 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
 import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 
 
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE;
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_PERFORMANCE_FLAGS;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+
 /**
 /**
  * Test dir operations on S3A.
  * Test dir operations on S3A.
  */
  */
 public class ITestS3AContractMkdir extends AbstractContractMkdirTest {
 public class ITestS3AContractMkdir extends AbstractContractMkdirTest {
 
 
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    removeBaseAndBucketOverrides(
+        conf,
+        FS_S3A_CREATE_PERFORMANCE,
+        FS_S3A_PERFORMANCE_FLAGS);
+    conf.set(FS_S3A_PERFORMANCE_FLAGS, "");
+    return conf;
+  }
+
   @Override
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
   protected AbstractFSContract createContract(Configuration conf) {
     return new S3AContract(conf);
     return new S3AContract(conf);

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

@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); 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
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.s3a;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE;
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_PERFORMANCE_FLAGS;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+
+/**
+ * Test mkdir operations on S3A with create performance mode.
+ */
+public class ITestS3AContractMkdirWithCreatePerf extends AbstractContractMkdirTest {
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration conf = super.createConfiguration();
+    removeBaseAndBucketOverrides(
+        conf,
+        FS_S3A_CREATE_PERFORMANCE,
+        FS_S3A_PERFORMANCE_FLAGS);
+    conf.setStrings(FS_S3A_PERFORMANCE_FLAGS,
+        "create,mkdir");
+    return conf;
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new S3AContract(conf);
+  }
+
+  @Test
+  public void testMkdirOverParentFile() throws Throwable {
+    describe("try to mkdir where a parent is a file, should pass");
+    FileSystem fs = getFileSystem();
+    Path path = methodPath();
+    byte[] dataset = dataset(1024, ' ', 'z');
+    createFile(getFileSystem(), path, false, dataset);
+    Path child = new Path(path, "child-to-mkdir");
+    boolean childCreated = fs.mkdirs(child);
+    assertTrue("Child dir is created", childCreated);
+    assertIsFile(path);
+    byte[] bytes = ContractTestUtils.readDataset(getFileSystem(), path, dataset.length);
+    ContractTestUtils.compareByteArrays(dataset, bytes, dataset.length);
+    assertPathExists("mkdir failed", child);
+    assertDeleted(child, true);
+  }
+
+}

+ 9 - 2
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextCreateMkdir.java

@@ -13,12 +13,15 @@
  */
  */
 package org.apache.hadoop.fs.s3a.fileContext;
 package org.apache.hadoop.fs.s3a.fileContext;
 
 
-import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
 import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.junit.Before;
 import org.junit.Before;
 
 
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE;
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_PERFORMANCE_FLAGS;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+
 /**
 /**
  * Extends FileContextCreateMkdirBaseTest for a S3a FileContext.
  * Extends FileContextCreateMkdirBaseTest for a S3a FileContext.
  */
  */
@@ -26,8 +29,12 @@ public class ITestS3AFileContextCreateMkdir
         extends FileContextCreateMkdirBaseTest {
         extends FileContextCreateMkdirBaseTest {
 
 
   @Before
   @Before
-  public void setUp() throws IOException, Exception {
+  public void setUp() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
+    removeBaseAndBucketOverrides(
+        conf,
+        FS_S3A_CREATE_PERFORMANCE,
+        FS_S3A_PERFORMANCE_FLAGS);
     fc = S3ATestUtils.createTestFileContext(conf);
     fc = S3ATestUtils.createTestFileContext(conf);
     super.setUp();
     super.setUp();
   }
   }

+ 67 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/fileContext/ITestS3AFileContextCreateMkdirCreatePerf.java

@@ -0,0 +1,67 @@
+/**
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * 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
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.s3a.fileContext;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest;
+import org.apache.hadoop.fs.s3a.S3ATestUtils;
+
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE;
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_PERFORMANCE_FLAGS;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+/**
+ * Extends FileContextCreateMkdirBaseTest for a S3a FileContext with
+ * create performance mode.
+ */
+public class ITestS3AFileContextCreateMkdirCreatePerf
+        extends FileContextCreateMkdirBaseTest {
+
+  @Before
+  public void setUp() throws Exception {
+    Configuration conf = new Configuration();
+    removeBaseAndBucketOverrides(
+        conf,
+        FS_S3A_CREATE_PERFORMANCE,
+        FS_S3A_PERFORMANCE_FLAGS);
+    conf.setStrings(FS_S3A_PERFORMANCE_FLAGS,
+        "mkdir");
+    fc = S3ATestUtils.createTestFileContext(conf);
+    super.setUp();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    if (fc != null) {
+      super.tearDown();
+    }
+  }
+
+  @Test
+  public void testMkdirRecursiveWithExistingFile() throws Exception {
+    intercept(
+        AssertionError.class,
+        MKDIR_FILE_PRESENT_ERROR,
+        "Dir creation should not have failed. "
+            + "Creation performance mode is expected "
+            + "to create dir without checking file "
+            + "status of parent dir.",
+        super::testMkdirRecursiveWithExistingFile);
+  }
+
+}