|
@@ -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);
|
|
|
+ });
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -627,6 +681,27 @@ public class ITestRestrictedReadAccess extends AbstractS3ATestBase {
|
|
|
return intercept(AccessDeniedException.class, eval);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * 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.
|
|
@@ -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)
|