You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2019/10/05 11:53:18 UTC

[hadoop] branch trunk updated: HADOOP-16626. S3A ITestRestrictedReadAccess fails without S3Guard.

This is an automated email from the ASF dual-hosted git repository.

stevel pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new b8086bf  HADOOP-16626. S3A ITestRestrictedReadAccess fails without S3Guard.
b8086bf is described below

commit b8086bf54d977199879a0f0dcff6058cdf56ded2
Author: Steve Loughran <st...@cloudera.com>
AuthorDate: Sat Oct 5 12:52:42 2019 +0100

    HADOOP-16626. S3A ITestRestrictedReadAccess fails without S3Guard.
    
    Contributed by Steve Loughran.
    
    Change-Id: Ife730b80057ddd43e919438cb5b2abbda990e636
---
 .../apache/hadoop/fs/contract/s3a/S3AContract.java |  25 +-
 .../apache/hadoop/fs/s3a/AbstractS3ATestBase.java  |  10 +-
 .../org/apache/hadoop/fs/s3a/S3ATestUtils.java     |  31 ++-
 .../fs/s3a/auth/ITestRestrictedReadAccess.java     | 257 +++++++++++++--------
 4 files changed, 224 insertions(+), 99 deletions(-)

diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java
index 3510a64..1311020 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/S3AContract.java
@@ -28,12 +28,35 @@ import org.apache.hadoop.fs.s3a.S3ATestUtils;
  */
 public class S3AContract extends AbstractBondedFSContract {
 
+  /**
+   * Test resource with the contract bindings used in the standard
+   * contract tests: {@value}.
+   */
   public static final String CONTRACT_XML = "contract/s3a.xml";
 
+  /**
+   * Instantiate, adding the s3a.xml contract file.
+   * This may force a reload of the entire configuration, so interferes with
+   * any code which has removed bucket overrides.
+   * @param conf configuration.
+   */
   public S3AContract(Configuration conf) {
+    this(conf, true);
+  }
+
+  /**
+   * Instantiate, optionally adding the s3a.xml contract file.
+   * This may force a reload of the entire configuration, so interferes with
+   * any code which has removed bucket overrides.
+   * @param conf configuration.
+   * @param addContractResource should the s3a.xml file be added?
+   */
+  public S3AContract(Configuration conf, boolean addContractResource) {
     super(conf);
     //insert the base features
-    addConfResource(CONTRACT_XML);
+    if (addContractResource) {
+      addConfResource(CONTRACT_XML);
+    }
   }
 
   @Override
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
index b05bbb4..a789eb5 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/AbstractS3ATestBase.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.fs.s3a;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.AbstractFSContract;
 import org.apache.hadoop.fs.contract.AbstractFSContractTestBase;
@@ -46,12 +47,16 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
 
   @Override
   protected AbstractFSContract createContract(Configuration conf) {
-    return new S3AContract(conf);
+    return new S3AContract(conf, false);
   }
 
   @Override
   public void setup() throws Exception {
     Thread.currentThread().setName("setup");
+    // force load the local FS -not because we want the FS, but we need all
+    // filesystems which add default configuration resources to do it before
+    // our tests start adding/removing options. See HADOOP-16626.
+    FileSystem.getLocal(new Configuration());
     super.setup();
   }
 
@@ -83,7 +88,8 @@ public abstract class AbstractS3ATestBase extends AbstractFSContractTestBase
    */
   @Override
   protected Configuration createConfiguration() {
-    return S3ATestUtils.prepareTestConfiguration(super.createConfiguration());
+    Configuration conf = super.createConfiguration();
+    return S3ATestUtils.prepareTestConfiguration(conf);
   }
 
   protected Configuration getConfiguration() {
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
index bd38b40..592c4be 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java
@@ -443,8 +443,7 @@ public final class S3ATestUtils {
    */
   public static void assumeS3GuardState(boolean shouldBeEnabled,
       Configuration originalConf) throws URISyntaxException {
-    boolean isEnabled = getTestPropertyBool(originalConf, TEST_S3GUARD_ENABLED,
-        originalConf.getBoolean(TEST_S3GUARD_ENABLED, false));
+    boolean isEnabled = isS3GuardTestPropertySet(originalConf);
     Assume.assumeThat("Unexpected S3Guard test state:"
             + " shouldBeEnabled=" + shouldBeEnabled
             + " and isEnabled=" + isEnabled,
@@ -463,12 +462,21 @@ public final class S3ATestUtils {
   }
 
   /**
+   * Is the test option for S3Guard set?
+   * @param conf configuration to examine.
+   * @return true if the config or system property turns s3guard tests on
+   */
+  public static boolean isS3GuardTestPropertySet(final Configuration conf) {
+    return getTestPropertyBool(conf, TEST_S3GUARD_ENABLED,
+        conf.getBoolean(TEST_S3GUARD_ENABLED, false));
+  }
+
+  /**
    * Conditionally set the S3Guard options from test properties.
    * @param conf configuration
    */
   public static void maybeEnableS3Guard(Configuration conf) {
-    if (getTestPropertyBool(conf, TEST_S3GUARD_ENABLED,
-        conf.getBoolean(TEST_S3GUARD_ENABLED, false))) {
+    if (isS3GuardTestPropertySet(conf)) {
       // S3Guard is enabled.
       boolean authoritative = getTestPropertyBool(conf,
           TEST_S3GUARD_AUTHORITATIVE,
@@ -758,8 +766,9 @@ public final class S3ATestUtils {
     for (String option : options) {
       final String stripped = option.substring("fs.s3a.".length());
       String target = bucketPrefix + stripped;
-      if (conf.get(target) != null) {
-        LOG.debug("Removing option {}", target);
+      String v = conf.get(target);
+      if (v != null) {
+        LOG.debug("Removing option {}; was {}", target, v);
         conf.unset(target);
       }
     }
@@ -1273,6 +1282,16 @@ public final class S3ATestUtils {
     return Boolean.valueOf(persists);
   }
 
+  /**
+   * Set the metadata store of a filesystem instance to the given
+   * store, via a package-private setter method.
+   * @param fs filesystem.
+   * @param ms metastore
+   */
+  public static void setMetadataStore(S3AFileSystem fs, MetadataStore ms) {
+    fs.setMetadataStore(ms);
+}
+
   public static void checkListingDoesNotContainPath(S3AFileSystem fs, Path filePath)
       throws IOException {
     final RemoteIterator<LocatedFileStatus> listIter =
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java
index a741cd6..0b03219 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestRestrictedReadAccess.java
@@ -20,13 +20,15 @@ package org.apache.hadoop.fs.s3a.auth;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.AccessDeniedException;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Optional;
 import java.util.concurrent.Callable;
 
 import org.assertj.core.api.Assertions;
+import org.assertj.core.api.Assumptions;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -41,14 +43,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
-import org.apache.hadoop.fs.s3a.Constants;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3ATestUtils;
 import org.apache.hadoop.fs.s3a.S3AUtils;
 import org.apache.hadoop.fs.s3a.Statistic;
-import org.apache.hadoop.fs.s3a.s3guard.LocalMetadataStore;
-import org.apache.hadoop.fs.s3a.s3guard.MetadataStore;
-import org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore;
+import org.apache.hadoop.fs.s3a.s3guard.DynamoDBMetadataStore;
 import org.apache.hadoop.mapred.LocatedFileStatusFetcher;
 import org.apache.hadoop.mapreduce.lib.input.InvalidInputException;
 
@@ -58,9 +57,12 @@ import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_ARN;
 import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE;
 import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.assume;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.assumeS3GuardState;
 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.isS3GuardTestPropertySet;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.lsR;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBucketOverrides;
 import static org.apache.hadoop.fs.s3a.auth.RoleModel.Effects;
 import static org.apache.hadoop.fs.s3a.auth.RoleModel.Statement;
@@ -81,8 +83,7 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
  * test for S3Guard + Auth to see how failures move around.
  * <ol>
  *   <li>Tests only run if an assumed role is provided.</li>
- *   <li>And the s3guard tests use the local metastore if
- *   there was not one already.</li>
+ *   <li>And the S3Guard tests require DynamoDB.</li>
  * </ol>
  * The tests are all bundled into one big test case.
  * From a purist unit test perspective, this is utterly wrong as it goes
@@ -106,6 +107,7 @@ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
  * To simplify maintenance, the operations tested are broken up into
  * their own methods, with fields used to share the restricted role and
  * created paths.
+ *
  */
 @SuppressWarnings("ThrowableNotThrown")
 @RunWith(Parameterized.class)
@@ -143,7 +145,9 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
   /**
    * Text used in files.
    */
-  public static final byte[] HELLO = "hello".getBytes(Charset.forName("UTF-8"));
+  public static final byte[] HELLO = "hello".getBytes(StandardCharsets.UTF_8);
+
+  private final boolean guardedInAuthMode;
 
   /**
    * Wildcard scan to find *.txt in the no-read directory.
@@ -209,20 +213,27 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
     this.name = name;
     this.s3guard = s3guard;
     this.authMode = authMode;
+    this.guardedInAuthMode = s3guard && authMode;
+
   }
 
   @Override
   public Configuration createConfiguration() {
     Configuration conf = super.createConfiguration();
     String bucketName = getTestBucketName(conf);
-    removeBucketOverrides(bucketName, conf,
-        S3_METADATA_STORE_IMPL,
+    // is s3guard enabled?
+    boolean guardedTestRun = isS3GuardTestPropertySet(conf);
+
+    // in a guarded test run, except for the special case of raw,
+    // all DDB settings are left alone.
+    removeBaseAndBucketOverrides(bucketName, conf,
         METADATASTORE_AUTHORITATIVE);
-    conf.setClass(Constants.S3_METADATA_STORE_IMPL,
-        s3guard ?
-            LocalMetadataStore.class
-            : NullMetadataStore.class,
-        MetadataStore.class);
+    removeBucketOverrides(bucketName, conf,
+        S3_METADATA_STORE_IMPL);
+    if (!s3guard) {
+      removeBaseAndBucketOverrides(bucketName, conf,
+          S3_METADATA_STORE_IMPL);
+    }
     conf.setBoolean(METADATASTORE_AUTHORITATIVE, authMode);
     disableFilesystemCaching(conf);
     return conf;
@@ -231,13 +242,21 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
   @Override
   public void setup() throws Exception {
     super.setup();
+    if (s3guard) {
+      // s3guard is required for those test runs where any of the
+      // guard options are set
+      assumeS3GuardState(true, getConfiguration());
+    }
     assumeRoleTests();
   }
 
   @Override
   public void teardown() throws Exception {
-    S3AUtils.closeAll(LOG, readonlyFS);
-    super.teardown();
+    try {
+      super.teardown();
+    } finally {
+      S3AUtils.closeAll(LOG, readonlyFS);
+    }
   }
 
   private void assumeRoleTests() {
@@ -295,6 +314,7 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
     describe("Setting up filesystem");
 
     S3AFileSystem realFS = getFileSystem();
+    verifyS3GuardSettings(realFS, "real filesystem");
 
     // avoiding the parameterization to steer clear of accidentally creating
     // patterns
@@ -328,6 +348,9 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
     subdir2File2 = new Path(subDir2, "subdir2File2.docx");
     createFile(realFS, subdir2File1, true, HELLO);
     createFile(realFS, subdir2File2, true, HELLO);
+    // execute a recursive list to make sure that S3Guard tables are always
+    // up to date
+    lsR(realFS, noReadDir, true);
 
     // create a role filesystem which does not have read access under a path
     // it still has write access, which can be explored in the final
@@ -341,6 +364,35 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
             .addActions(S3_ALL_GET)
             .addResources(directory(noReadDir)));
     readonlyFS = (S3AFileSystem) basePath.getFileSystem(roleConfig);
+    verifyS3GuardSettings(readonlyFS, "readonly");
+  }
+
+  /**
+   * Verify that the FS (real or restricted) meets the
+   * requirement of the test.
+   * S3Guard tests are skipped if the (default) store is not
+   * a DDB store consistent across all FS instances.
+   * The raw tests fail if somehow the FS does still have a S3Guard metastore.
+   * @param fs filesystem
+   * @param storeType store role for error messages.
+   */
+  protected void verifyS3GuardSettings(final S3AFileSystem fs,
+      final String storeType) {
+    if (s3guard) {
+      Assumptions.assumeThat(fs.getMetadataStore())
+          .describedAs("Metadata store in "
+                  + storeType
+                  + " fs: %s",
+              fs.getMetadataStore())
+          .isInstanceOf(DynamoDBMetadataStore.class);
+    } else {
+      Assertions.assertThat(fs.hasMetadataStore())
+          .describedAs("Metadata store in "
+                  + storeType
+                  + " fs: %s",
+              fs.getMetadataStore())
+          .isFalse();
+    }
   }
 
   /**
@@ -355,53 +407,50 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
     //   - LIST path -> list results
     // Because the client has list access, this succeeds
     readonlyFS.listStatus(basePath);
+    lsR(readonlyFS, basePath, true);
 
-    // this is HEAD + "/" on S3; get on S3Guard auth
-    readonlyFS.listStatus(emptyDir);
+
+    // this is HEAD + "/" on S3; get on S3Guard auth when the path exists,
+    accessDeniedIf(!s3guard, () ->
+        readonlyFS.listStatus(emptyDir));
 
     // a recursive list of the no-read-directory works because
     // there is no directory marker, it becomes a LIST call.
     lsR(readonlyFS, noReadDir, true);
 
-    // similarly, a getFileStatus ends up being a list and generating
-    // a file status marker.
+    // similarly, a getFileStatus ends up being a list of the path
+    // and so working.
     readonlyFS.getFileStatus(noReadDir);
 
-    // empty dir checks work!
-    readonlyFS.getFileStatus(emptyDir);
+    // empty dir checks work when guarded because even in non-auth mode
+    // there are no checks for directories being out of date
+    // without S3, the HEAD path + "/" is blocked
+    accessDeniedIf(!s3guard, () ->
+        readonlyFS.getFileStatus(emptyDir));
 
     // now look at a file; the outcome depends on the mode.
-    if (authMode) {
-      // auth mode doesn't check S3, so no failure
-      readonlyFS.getFileStatus(subdirFile);
-    } else {
-      accessDenied(() ->
-          readonlyFS.getFileStatus(subdirFile));
-    }
+    accessDeniedIf(!guardedInAuthMode, () ->
+        readonlyFS.getFileStatus(subdirFile));
 
     // irrespective of mode, the attempt to read the data will fail.
     // the only variable is where the failure occurs
     accessDenied(() ->
         ContractTestUtils.readUTF8(readonlyFS, subdirFile, HELLO.length));
 
-    // the empty file is interesting
-    if (!authMode) {
-      // non-auth mode, it fails at some point in the opening process.
-      // due to a HEAD being called on the object
-      accessDenied(() ->
-          ContractTestUtils.readUTF8(readonlyFS, emptyFile, 0));
-    } else {
-      // auth mode doesn't check the store.
-      // Furthermore, because it knows the file length is zero,
-      // it returns -1 without even opening the file.
-      // This means that permissions on the file do not get checked.
-      // See: HADOOP-16464.
-      try (FSDataInputStream is = readonlyFS.open(emptyFile)) {
+    // the empty file is interesting.
+    // auth mode doesn't check the store.
+    // Furthermore, because it knows the file length is zero,
+    // it returns -1 without even opening the file.
+    // This means that permissions on the file do not get checked.
+    // See: HADOOP-16464.
+    Optional<FSDataInputStream> optIn = accessDeniedIf(
+        !guardedInAuthMode, () -> readonlyFS.open(emptyFile));
+    if (optIn.isPresent()) {
+      try (FSDataInputStream is = optIn.get()) {
         Assertions.assertThat(is.read())
             .describedAs("read of empty file")
             .isEqualTo(-1);
       }
-      readonlyFS.getFileStatus(subdirFile);
     }
   }
 
@@ -414,27 +463,31 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
     // baseline: the real filesystem on a subdir
     globFS(getFileSystem(), subdirFile, null, false, 1);
     // a file fails if not in auth mode
-    globFS(readonlyFS, subdirFile, null, !authMode, 1);
+    globFS(readonlyFS, subdirFile, null, !guardedInAuthMode, 1);
     // empty directories don't fail.
-    assertStatusPathEquals(emptyDir,
-        globFS(readonlyFS, emptyDir, null, false, 1));
+    FileStatus[] st = globFS(readonlyFS, emptyDir, null, !s3guard, 1);
+    if (s3guard) {
+      assertStatusPathEquals(emptyDir, st);
+    }
 
-    FileStatus[] st = globFS(readonlyFS,
+    st = globFS(readonlyFS,
         noReadWildcard,
-        null, false, 2);
-    Assertions.assertThat(st)
-        .extracting(FileStatus::getPath)
-        .containsExactlyInAnyOrder(subdirFile, subdir2File1);
+        null, !s3guard, 2);
+    if (s3guard) {
+      Assertions.assertThat(st)
+          .extracting(FileStatus::getPath)
+          .containsExactlyInAnyOrder(subdirFile, subdir2File1);
+    }
 
     // there is precisely one .docx file (subdir2File2.docx)
     globFS(readonlyFS,
         new Path(noReadDir, "*/*.docx"),
-        null, false, 1);
+        null, !s3guard, 1);
 
     // there are no .doc files.
     globFS(readonlyFS,
         new Path(noReadDir, "*/*.doc"),
-        null, false, 0);
+        null, !s3guard, 0);
     globFS(readonlyFS, noReadDir,
         EVERYTHING, false, 1);
     // and a filter without any wildcarded pattern only finds
@@ -461,15 +514,17 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
             true,
             HIDDEN_FILE_FILTER,
             true);
-    Assertions.assertThat(fetcher.getFileStatuses())
-        .describedAs("result of located scan")
-        .flatExtracting(FileStatus::getPath)
-        .containsExactlyInAnyOrder(
-            emptyFile,
-            subdirFile,
-            subdir2File1,
-            subdir2File2);
-
+    accessDeniedIf(!s3guard,
+        () -> fetcher.getFileStatuses())
+        .ifPresent(stats -> {
+          Assertions.assertThat(stats)
+              .describedAs("result of located scan").flatExtracting(FileStatus::getPath)
+              .containsExactlyInAnyOrder(
+                  emptyFile,
+                  subdirFile,
+                  subdir2File1,
+                  subdir2File2);
+        });
   }
 
   /**
@@ -481,18 +536,22 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
     int threads = 4;
     describe("LocatedFileStatusFetcher with %d", threads);
     roleConfig.setInt(LIST_STATUS_NUM_THREADS, threads);
-    LocatedFileStatusFetcher fetcher2 =
+    LocatedFileStatusFetcher fetcher =
         new LocatedFileStatusFetcher(
             roleConfig,
             new Path[]{noReadWildcard},
             true,
             EVERYTHING,
             true);
-    Assertions.assertThat(fetcher2.getFileStatuses())
-        .describedAs("result of located scan")
-        .isNotNull()
-        .flatExtracting(FileStatus::getPath)
-        .containsExactlyInAnyOrder(subdirFile, subdir2File1);
+    accessDeniedIf(!s3guard,
+        () -> fetcher.getFileStatuses())
+        .ifPresent(stats -> {
+          Assertions.assertThat(stats)
+              .describedAs("result of located scan")
+              .isNotNull()
+              .flatExtracting(FileStatus::getPath)
+              .containsExactlyInAnyOrder(subdirFile, subdir2File1);
+        });
   }
 
   /**
@@ -502,27 +561,22 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
     // pass in a file as the base of the scan.
     describe("LocatedFileStatusFetcher with file %s", subdirFile);
     roleConfig.setInt(LIST_STATUS_NUM_THREADS, 16);
-    try {
-      Iterable<FileStatus> fetched = new LocatedFileStatusFetcher(
-          roleConfig,
-          new Path[]{subdirFile},
-          true,
-          TEXT_FILE,
-          true).getFileStatuses();
-      // when not in auth mode, the HEAD request MUST have failed.
-      failif(!authMode, "LocatedFileStatusFetcher(" + subdirFile + ")"
-          + " should have failed");
-      // and in auth mode, the file MUST have been found.
-      Assertions.assertThat(fetched)
-          .describedAs("result of located scan")
-          .isNotNull()
-          .flatExtracting(FileStatus::getPath)
-          .containsExactly(subdirFile);
-    } catch (AccessDeniedException e) {
-      // we require the HEAD request to fail with access denied in non-auth
-      // mode, but not in auth mode.
-      failif(authMode, "LocatedFileStatusFetcher(" + subdirFile + ")", e);
-    }
+    LocatedFileStatusFetcher fetcher
+        = new LocatedFileStatusFetcher(
+        roleConfig,
+        new Path[]{subdirFile},
+        true,
+        TEXT_FILE,
+        true);
+    accessDeniedIf(!guardedInAuthMode,
+        () -> fetcher.getFileStatuses())
+        .ifPresent(stats -> {
+          Assertions.assertThat(stats)
+              .describedAs("result of located scan")
+              .isNotNull()
+              .flatExtracting(FileStatus::getPath)
+              .containsExactly(subdirFile);
+        });
   }
 
   /**
@@ -628,6 +682,27 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
   }
 
   /**
+   * Conditionally expect an operation to fail with an AccessDeniedException.
+   * @param condition the condition which must be true for access to be denied
+   * @param eval closure to evaluate.
+   * @param <T> type of callable
+   * @return the return value if the call succeeded
+   * and did not return null.
+   * @throws Exception any unexpected exception
+   */
+  protected <T> Optional<T> accessDeniedIf(
+      final boolean condition,
+      final Callable<T> eval)
+      throws Exception {
+    if (condition) {
+      intercept(AccessDeniedException.class, eval);
+      return Optional.empty();
+    } else {
+      return Optional.ofNullable(eval.call());
+    }
+  }
+
+  /**
    * Assert that a status array has exactly one element and its
    * value is as expected.
    * @param expected expected path
@@ -689,6 +764,8 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
       failif(!expectAuthFailure, "Access denied in glob of " + path,
           e);
       return null;
+    } catch (IOException | RuntimeException e) {
+      throw new AssertionError("Other exception raised in glob:" + e, e);
     }
     if (expectedCount < 0) {
       Assertions.assertThat(st)


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org