keys
= buildDeleteRequest(
new String[]{
- fs.pathToKey(csvPath),
+ fs.pathToKey(path),
"missing-key.csv"
});
MultiObjectDeleteException ex = intercept(
@@ -193,10 +184,10 @@ public void testMultiObjectDeleteNoPermissions() throws Throwable {
final String undeletedFiles = undeleted.stream()
.map(Path::toString)
.collect(Collectors.joining(", "));
- failIf(undeleted.size() != 2,
- "undeleted list size wrong: " + undeletedFiles,
- ex);
- assertTrue("no CSV in " +undeletedFiles, undeleted.contains(csvPath));
+ Assertions.assertThat(undeleted)
+ .describedAs("undeleted files")
+ .hasSize(2)
+ .contains(path);
}
/**
@@ -205,12 +196,12 @@ public void testMultiObjectDeleteNoPermissions() throws Throwable {
*/
@Test
public void testSingleObjectDeleteNoPermissionsTranslated() throws Throwable {
- describe("Delete the landsat CSV file and expect it to fail");
- Path csvPath = maybeGetCsvPath();
- S3AFileSystem fs = (S3AFileSystem) csvPath.getFileSystem(
+ describe("Delete the external file and expect it to fail");
+ Path path = requireDefaultExternalData(getConfiguration());
+ S3AFileSystem fs = (S3AFileSystem) path.getFileSystem(
getConfiguration());
AccessDeniedException aex = intercept(AccessDeniedException.class,
- () -> fs.delete(csvPath, false));
+ () -> fs.delete(path, false));
Throwable cause = aex.getCause();
failIf(cause == null, "no nested exception", aex);
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java
index fe00cb5b0e31b..ce962483d5840 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3APrefetchingCacheFiles.java
@@ -19,8 +19,9 @@
package org.apache.hadoop.fs.s3a;
import java.io.File;
-import java.net.URI;
+import java.util.UUID;
+import org.assertj.core.api.Assertions;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
@@ -30,15 +31,16 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
import static org.apache.hadoop.fs.s3a.Constants.BUFFER_DIR;
-import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_DEFAULT_SIZE;
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_BLOCK_SIZE_KEY;
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
+import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getExternalData;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
/**
@@ -49,11 +51,21 @@ public class ITestS3APrefetchingCacheFiles extends AbstractS3ACostTest {
private static final Logger LOG =
LoggerFactory.getLogger(ITestS3APrefetchingCacheFiles.class);
+ /** use a small file size so small source files will still work. */
+ public static final int BLOCK_SIZE = 128 * 1024;
+
+ public static final int PREFETCH_OFFSET = 10240;
+
private Path testFile;
+
+ /** The FS with the external file. */
private FileSystem fs;
+
private int prefetchBlockSize;
private Configuration conf;
+ private String bufferDir;
+
public ITestS3APrefetchingCacheFiles() {
super(true);
}
@@ -63,35 +75,31 @@ public void setUp() throws Exception {
super.setup();
// Sets BUFFER_DIR by calling S3ATestUtils#prepareTestConfiguration
conf = createConfiguration();
- String testFileUri = S3ATestUtils.getCSVTestFile(conf);
- testFile = new Path(testFileUri);
- prefetchBlockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, PREFETCH_BLOCK_DEFAULT_SIZE);
- fs = getFileSystem();
- fs.initialize(new URI(testFileUri), conf);
+ testFile = getExternalData(conf);
+ prefetchBlockSize = conf.getInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE);
+ fs = FileSystem.get(testFile.toUri(), conf);
}
@Override
public Configuration createConfiguration() {
Configuration configuration = super.createConfiguration();
S3ATestUtils.removeBaseAndBucketOverrides(configuration, PREFETCH_ENABLED_KEY);
- S3ATestUtils.removeBaseAndBucketOverrides(configuration, PREFETCH_BLOCK_SIZE_KEY);
configuration.setBoolean(PREFETCH_ENABLED_KEY, true);
+ // use a small block size unless explicitly set in the test config.
+ configuration.setInt(PREFETCH_BLOCK_SIZE_KEY, BLOCK_SIZE);
+ // patch buffer dir with a unique path for test isolation.
+ final String bufferDirBase = configuration.get(BUFFER_DIR);
+ bufferDir = bufferDirBase + "/" + UUID.randomUUID();
+ configuration.set(BUFFER_DIR, bufferDir);
return configuration;
}
@Override
public synchronized void teardown() throws Exception {
super.teardown();
- File tmpFileDir = new File(conf.get(BUFFER_DIR));
- File[] tmpFiles = tmpFileDir.listFiles();
- if (tmpFiles != null) {
- for (File filePath : tmpFiles) {
- String path = filePath.getPath();
- if (path.endsWith(".bin") && path.contains("fs-cache-")) {
- filePath.delete();
- }
- }
+ if (bufferDir != null) {
+ new File(bufferDir).delete();
}
cleanupWithLogger(LOG, fs);
fs = null;
@@ -111,34 +119,35 @@ public void testCacheFileExistence() throws Throwable {
try (FSDataInputStream in = fs.open(testFile)) {
byte[] buffer = new byte[prefetchBlockSize];
- in.read(buffer, 0, prefetchBlockSize - 10240);
- in.seek(prefetchBlockSize * 2);
- in.read(buffer, 0, prefetchBlockSize);
+ // read a bit less than a block
+ in.readFully(0, buffer, 0, prefetchBlockSize - PREFETCH_OFFSET);
+ // read at least some of a second block
+ in.read(prefetchBlockSize * 2, buffer, 0, prefetchBlockSize);
+
File tmpFileDir = new File(conf.get(BUFFER_DIR));
- assertTrue("The dir to keep cache files must exist", tmpFileDir.exists());
+ final LocalFileSystem localFs = FileSystem.getLocal(conf);
+ Path bufferDirPath = new Path(tmpFileDir.toURI());
+ ContractTestUtils.assertIsDirectory(localFs, bufferDirPath);
File[] tmpFiles = tmpFileDir
.listFiles((dir, name) -> name.endsWith(".bin") && name.contains("fs-cache-"));
- boolean isCacheFileForBlockFound = tmpFiles != null && tmpFiles.length > 0;
- if (!isCacheFileForBlockFound) {
- LOG.warn("No cache files found under " + tmpFileDir);
- }
- assertTrue("File to cache block data must exist", isCacheFileForBlockFound);
+ Assertions.assertThat(tmpFiles)
+ .describedAs("Cache files not found under %s", tmpFileDir)
+ .isNotEmpty();
+
for (File tmpFile : tmpFiles) {
Path path = new Path(tmpFile.getAbsolutePath());
- try (FileSystem localFs = FileSystem.getLocal(conf)) {
- FileStatus stat = localFs.getFileStatus(path);
- ContractTestUtils.assertIsFile(path, stat);
- assertEquals("File length not matching with prefetchBlockSize", prefetchBlockSize,
- stat.getLen());
- assertEquals("User permissions should be RW", FsAction.READ_WRITE,
- stat.getPermission().getUserAction());
- assertEquals("Group permissions should be NONE", FsAction.NONE,
- stat.getPermission().getGroupAction());
- assertEquals("Other permissions should be NONE", FsAction.NONE,
- stat.getPermission().getOtherAction());
- }
+ FileStatus stat = localFs.getFileStatus(path);
+ ContractTestUtils.assertIsFile(path, stat);
+ assertEquals("File length not matching with prefetchBlockSize", prefetchBlockSize,
+ stat.getLen());
+ assertEquals("User permissions should be RW", FsAction.READ_WRITE,
+ stat.getPermission().getUserAction());
+ assertEquals("Group permissions should be NONE", FsAction.NONE,
+ stat.getPermission().getGroupAction());
+ assertEquals("Other permissions should be NONE", FsAction.NONE,
+ stat.getPermission().getOtherAction());
}
}
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
index f2e0223adb1d9..af04c2fa634e9 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java
@@ -111,14 +111,16 @@ public interface S3ATestConstants {
String KEY_CSVTEST_FILE = S3A_SCALE_TEST + "csvfile";
/**
- * The landsat bucket: {@value}.
+ * Default path for the multi MB test file: {@value}.
+ * @deprecated retrieve via {@link PublicDatasetTestUtils}.
*/
- String LANDSAT_BUCKET = "s3a://landsat-pds/";
+ @Deprecated
+ String DEFAULT_CSVTEST_FILE = PublicDatasetTestUtils.DEFAULT_EXTERNAL_FILE;
/**
- * Default path for the multi MB test file: {@value}.
+ * Example path for unit tests; this is never accessed: {@value}.
*/
- String DEFAULT_CSVTEST_FILE = LANDSAT_BUCKET + "scene_list.gz";
+ String UNIT_TEST_EXAMPLE_PATH = "s3a://example/data/";
/**
* Configuration key for an existing object in a requester pays bucket: {@value}.
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 e7ea920d8a0a0..d91bbe4656ec2 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
@@ -105,6 +105,8 @@
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit;
import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletion;
import static org.apache.hadoop.fs.s3a.impl.S3ExpressStorage.STORE_CAPABILITY_S3_EXPRESS_STORAGE;
+import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getExternalData;
+import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.requireDefaultExternalDataFile;
import static org.apache.hadoop.test.GenericTestUtils.buildPaths;
import static org.apache.hadoop.util.Preconditions.checkNotNull;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CREDENTIAL_PROVIDER_PATH;
@@ -405,22 +407,22 @@ public static String getTestProperty(Configuration conf,
* Get the test CSV file; assume() that it is not empty.
* @param conf test configuration
* @return test file.
+ * @deprecated Retained only to assist cherrypicking patches
*/
+ @Deprecated
public static String getCSVTestFile(Configuration conf) {
- String csvFile = conf
- .getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
- Assume.assumeTrue("CSV test file is not the default",
- isNotEmpty(csvFile));
- return csvFile;
+ return getExternalData(conf).toUri().toString();
}
/**
* Get the test CSV path; assume() that it is not empty.
* @param conf test configuration
* @return test file as a path.
+ * @deprecated Retained only to assist cherrypicking patches
*/
+ @Deprecated
public static Path getCSVTestPath(Configuration conf) {
- return new Path(getCSVTestFile(conf));
+ return getExternalData(conf);
}
/**
@@ -429,12 +431,11 @@ public static Path getCSVTestPath(Configuration conf) {
* read only).
* @return test file.
* @param conf test configuration
+ * @deprecated Retained only to assist cherrypicking patches
*/
+ @Deprecated
public static String getLandsatCSVFile(Configuration conf) {
- String csvFile = getCSVTestFile(conf);
- Assume.assumeTrue("CSV test file is not the default",
- DEFAULT_CSVTEST_FILE.equals(csvFile));
- return csvFile;
+ return requireDefaultExternalDataFile(conf);
}
/**
* Get the test CSV file; assume() that it is not modified (i.e. we haven't
@@ -442,9 +443,11 @@ public static String getLandsatCSVFile(Configuration conf) {
* read only).
* @param conf test configuration
* @return test file as a path.
+ * @deprecated Retained only to assist cherrypicking patches
*/
+ @Deprecated
public static Path getLandsatCSVPath(Configuration conf) {
- return new Path(getLandsatCSVFile(conf));
+ return getExternalData(conf);
}
/**
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
index 41e421b058117..8358570d83ace 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AAWSCredentialsProvider.java
@@ -54,37 +54,34 @@
import org.apache.hadoop.fs.s3a.auth.NoAuthWithAWSException;
import org.apache.hadoop.fs.s3a.auth.delegation.CountInvocationsProvider;
import org.apache.hadoop.fs.s3a.impl.InstantiationIOException;
+import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.util.Sets;
import static org.apache.hadoop.fs.s3a.Constants.ASSUMED_ROLE_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER_MAPPING;
-import static org.apache.hadoop.fs.s3a.S3ATestConstants.DEFAULT_CSVTEST_FILE;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.authenticationContains;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.buildClassListString;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.getCSVTestPath;
import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.STANDARD_AWS_PROVIDERS;
import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.buildAWSProviderList;
import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.createAWSCredentialProviderList;
import static org.apache.hadoop.fs.s3a.impl.InstantiationIOException.DOES_NOT_IMPLEMENT;
+import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getExternalData;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
/**
* Unit tests for {@link Constants#AWS_CREDENTIALS_PROVIDER} logic.
*/
-public class TestS3AAWSCredentialsProvider {
+public class TestS3AAWSCredentialsProvider extends AbstractS3ATestBase {
/**
- * URI of the landsat images.
+ * URI of the test file: this must be anonymously accessible.
+ * As these are unit tests no actual connection to the store is made.
*/
private static final URI TESTFILE_URI = new Path(
- DEFAULT_CSVTEST_FILE).toUri();
+ PublicDatasetTestUtils.DEFAULT_EXTERNAL_FILE).toUri();
private static final Logger LOG = LoggerFactory.getLogger(TestS3AAWSCredentialsProvider.class);
@@ -127,7 +124,7 @@ public void testInstantiationChain() throws Throwable {
TemporaryAWSCredentialsProvider.NAME
+ ", \t" + SimpleAWSCredentialsProvider.NAME
+ " ,\n " + AnonymousAWSCredentialsProvider.NAME);
- Path testFile = getCSVTestPath(conf);
+ Path testFile = getExternalData(conf);
AWSCredentialProviderList list = createAWSCredentialProviderList(
testFile.toUri(), conf);
@@ -586,7 +583,7 @@ protected AwsCredentials createCredentials(Configuration config) throws IOExcept
@Test
public void testConcurrentAuthentication() throws Throwable {
Configuration conf = createProviderConfiguration(SlowProvider.class.getName());
- Path testFile = getCSVTestPath(conf);
+ Path testFile = getExternalData(conf);
AWSCredentialProviderList list = createAWSCredentialProviderList(testFile.toUri(), conf);
@@ -656,7 +653,7 @@ protected AwsCredentials createCredentials(Configuration config) throws IOExcept
@Test
public void testConcurrentAuthenticationError() throws Throwable {
Configuration conf = createProviderConfiguration(ErrorProvider.class.getName());
- Path testFile = getCSVTestPath(conf);
+ Path testFile = getExternalData(conf);
AWSCredentialProviderList list = createAWSCredentialProviderList(testFile.toUri(), conf);
ErrorProvider provider = (ErrorProvider) list.getProviders().get(0);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/adapter/TestV1CredentialsProvider.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/adapter/TestV1CredentialsProvider.java
index b0e1b57d75471..48c1f5034c95b 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/adapter/TestV1CredentialsProvider.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/adapter/TestV1CredentialsProvider.java
@@ -39,9 +39,9 @@
import org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider;
import org.apache.hadoop.fs.s3a.auth.IAMInstanceCredentialsProvider;
import org.apache.hadoop.fs.s3a.impl.InstantiationIOException;
+import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
import static org.apache.hadoop.fs.s3a.Constants.AWS_CREDENTIALS_PROVIDER;
-import static org.apache.hadoop.fs.s3a.S3ATestConstants.DEFAULT_CSVTEST_FILE;
import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.ANONYMOUS_CREDENTIALS_V1;
import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.EC2_CONTAINER_CREDENTIALS_V1;
import static org.apache.hadoop.fs.s3a.auth.CredentialProviderListFactory.ENVIRONMENT_CREDENTIALS_V1;
@@ -56,10 +56,10 @@
public class TestV1CredentialsProvider {
/**
- * URI of the landsat images.
+ * URI of the test file.
*/
private static final URI TESTFILE_URI = new Path(
- DEFAULT_CSVTEST_FILE).toUri();
+ PublicDatasetTestUtils.DEFAULT_EXTERNAL_FILE).toUri();
private static final Logger LOG = LoggerFactory.getLogger(TestV1CredentialsProvider.class);
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
index f2a4ea5534f8a..a7ccc92e133c8 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
@@ -46,7 +46,6 @@
import org.apache.hadoop.fs.s3a.AWSBadRequestException;
import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.s3a.S3ATestConstants;
import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider;
import org.apache.hadoop.fs.s3a.commit.CommitConstants;
import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
@@ -68,6 +67,7 @@
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden;
import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec;
+import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.requireAnonymousDataPath;
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
@@ -115,7 +115,7 @@ protected Configuration createConfiguration() {
public void setup() throws Exception {
super.setup();
assumeRoleTests();
- uri = new URI(S3ATestConstants.DEFAULT_CSVTEST_FILE);
+ uri = requireAnonymousDataPath(getConfiguration()).toUri();
}
@Override
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestDelegatedMRJob.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestDelegatedMRJob.java
index d5d62f2cae92c..ba9746358c575 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestDelegatedMRJob.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestDelegatedMRJob.java
@@ -58,6 +58,8 @@
import static org.apache.hadoop.fs.s3a.auth.delegation.DelegationConstants.*;
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.assertSecurityEnabled;
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.closeUserFileSystems;
+import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.getOrcData;
+import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.requireAnonymousDataPath;
/**
* Submit a job with S3 delegation tokens.
@@ -106,10 +108,17 @@ public class ITestDelegatedMRJob extends AbstractDelegationIT {
private Path destPath;
- private static final Path EXTRA_JOB_RESOURCE_PATH
- = new Path("s3a://osm-pds/planet/planet-latest.orc");
+ /**
+ * Path of the extra job resource; set up in
+ * {@link #createConfiguration()}.
+ */
+ private Path extraJobResourcePath;
- public static final URI jobResource = EXTRA_JOB_RESOURCE_PATH.toUri();
+ /**
+ * URI of the extra job resource; set up in
+ * {@link #createConfiguration()}.
+ */
+ private URI jobResourceUri;
/**
* Test array for parameterized test runs.
@@ -161,7 +170,9 @@ protected YarnConfiguration createConfiguration() {
conf.setInt(YarnConfiguration.RESOURCEMANAGER_CONNECT_RETRY_INTERVAL_MS,
10_000);
- String host = jobResource.getHost();
+ extraJobResourcePath = getOrcData(conf);
+ jobResourceUri = extraJobResourcePath.toUri();
+ String host = jobResourceUri.getHost();
// and fix to the main endpoint if the caller has moved
conf.set(
String.format("fs.s3a.bucket.%s.endpoint", host), "");
@@ -229,9 +240,9 @@ protected int getTestTimeoutMillis() {
@Test
public void testCommonCrawlLookup() throws Throwable {
- FileSystem resourceFS = EXTRA_JOB_RESOURCE_PATH.getFileSystem(
+ FileSystem resourceFS = extraJobResourcePath.getFileSystem(
getConfiguration());
- FileStatus status = resourceFS.getFileStatus(EXTRA_JOB_RESOURCE_PATH);
+ FileStatus status = resourceFS.getFileStatus(extraJobResourcePath);
LOG.info("Extra job resource is {}", status);
assertTrue("Not encrypted: " + status, status.isEncrypted());
}
@@ -241,9 +252,9 @@ public void testJobSubmissionCollectsTokens() throws Exception {
describe("Mock Job test");
JobConf conf = new JobConf(getConfiguration());
- // the input here is the landsat file; which lets
+ // the input here is the external file; which lets
// us differentiate source URI from dest URI
- Path input = new Path(DEFAULT_CSVTEST_FILE);
+ Path input = requireAnonymousDataPath(getConfiguration());
final FileSystem sourceFS = input.getFileSystem(conf);
@@ -272,7 +283,7 @@ public void testJobSubmissionCollectsTokens() throws Exception {
// This is to actually stress the terasort code for which
// the yarn ResourceLocalizationService was having problems with
// fetching resources from.
- URI partitionUri = new URI(EXTRA_JOB_RESOURCE_PATH.toString() +
+ URI partitionUri = new URI(extraJobResourcePath.toString() +
"#_partition.lst");
job.addCacheFile(partitionUri);
@@ -302,7 +313,7 @@ public void testJobSubmissionCollectsTokens() throws Exception {
// look up the destination token
lookupToken(submittedCredentials, fs.getUri(), tokenKind);
lookupToken(submittedCredentials,
- EXTRA_JOB_RESOURCE_PATH.getFileSystem(conf).getUri(), tokenKind);
+ extraJobResourcePath.getFileSystem(conf).getUri(), tokenKind);
}
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestRoleDelegationInFilesystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestRoleDelegationInFilesystem.java
index 511b813475954..08dba4b798214 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestRoleDelegationInFilesystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestRoleDelegationInFilesystem.java
@@ -53,8 +53,7 @@ public Text getTokenKind() {
/**
* This verifies that the granted credentials only access the target bucket
- * by using the credentials in a new S3 client to query the AWS-owned landsat
- * bucket.
+ * by using the credentials in a new S3 client to query the public data bucket.
* @param delegatedFS delegated FS with role-restricted access.
* @throws Exception failure
*/
@@ -62,7 +61,7 @@ public Text getTokenKind() {
protected void verifyRestrictedPermissions(final S3AFileSystem delegatedFS)
throws Exception {
intercept(AccessDeniedException.class,
- () -> readLandsatMetadata(delegatedFS));
+ () -> readExternalDatasetMetadata(delegatedFS));
}
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFilesystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFilesystem.java
index 808de4769c986..b2be0bc7d75ed 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFilesystem.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/ITestSessionDelegationInFilesystem.java
@@ -79,6 +79,7 @@
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.ALICE;
import static org.apache.hadoop.fs.s3a.auth.delegation.MiniKerberizedHadoopCluster.assertSecurityEnabled;
import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.lookupS3ADelegationToken;
+import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.requireAnonymousDataPath;
import static org.apache.hadoop.test.LambdaTestUtils.doAs;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
import static org.hamcrest.Matchers.containsString;
@@ -344,7 +345,7 @@ public void testDelegatedFileSystem() throws Throwable {
// TODO: Check what should happen here. Calling headObject() on the root path fails in V2,
// with the error that key cannot be empty.
// fs.getObjectMetadata(new Path("/"));
- readLandsatMetadata(fs);
+ readExternalDatasetMetadata(fs);
URI uri = fs.getUri();
// create delegation tokens from the test suites FS.
@@ -463,13 +464,13 @@ protected void executeDelegatedFSOperations(final S3AFileSystem delegatedFS,
}
/**
- * Session tokens can read the landsat bucket without problems.
+ * Session tokens can read the external bucket without problems.
* @param delegatedFS delegated FS
* @throws Exception failure
*/
protected void verifyRestrictedPermissions(final S3AFileSystem delegatedFS)
throws Exception {
- readLandsatMetadata(delegatedFS);
+ readExternalDatasetMetadata(delegatedFS);
}
@Test
@@ -582,7 +583,7 @@ public void testDelegationBindingMismatch2() throws Throwable {
/**
* This verifies that the granted credentials only access the target bucket
- * by using the credentials in a new S3 client to query the AWS-owned landsat
+ * by using the credentials in a new S3 client to query the external
* bucket.
* @param delegatedFS delegated FS with role-restricted access.
* @throws AccessDeniedException if the delegated FS's credentials can't
@@ -590,17 +591,17 @@ public void testDelegationBindingMismatch2() throws Throwable {
* @return result of the HEAD
* @throws Exception failure
*/
- protected HeadBucketResponse readLandsatMetadata(final S3AFileSystem delegatedFS)
+ protected HeadBucketResponse readExternalDatasetMetadata(final S3AFileSystem delegatedFS)
throws Exception {
AWSCredentialProviderList testingCreds
= delegatedFS.getS3AInternals().shareCredentials("testing");
- URI landsat = new URI(DEFAULT_CSVTEST_FILE);
+ URI external = requireAnonymousDataPath(getConfiguration()).toUri();
DefaultS3ClientFactory factory
= new DefaultS3ClientFactory();
Configuration conf = delegatedFS.getConf();
factory.setConf(conf);
- String host = landsat.getHost();
+ String host = external.getHost();
S3ClientFactory.S3ClientCreationParameters parameters = null;
parameters = new S3ClientFactory.S3ClientCreationParameters()
.withCredentialSet(testingCreds)
@@ -609,7 +610,7 @@ protected HeadBucketResponse readLandsatMetadata(final S3AFileSystem delegatedFS
.newStatisticsFromAwsSdk())
.withUserAgentSuffix("ITestSessionDelegationInFilesystem");
- S3Client s3 = factory.createS3Client(landsat, parameters);
+ S3Client s3 = factory.createS3Client(external, parameters);
return Invoker.once("HEAD", host,
() -> s3.headBucket(b -> b.bucket(host)));
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/TestS3ADelegationTokenSupport.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/TestS3ADelegationTokenSupport.java
index 992643ff8ce98..af306cc5a9a5f 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/TestS3ADelegationTokenSupport.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/delegation/TestS3ADelegationTokenSupport.java
@@ -24,10 +24,10 @@
import org.junit.Test;
import org.apache.hadoop.fs.s3a.S3AEncryptionMethods;
-import org.apache.hadoop.fs.s3a.S3ATestConstants;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding;
import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials;
+import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager;
@@ -44,11 +44,11 @@
*/
public class TestS3ADelegationTokenSupport {
- private static URI landsatUri;
+ private static URI externalUri;
@BeforeClass
public static void classSetup() throws Exception {
- landsatUri = new URI(S3ATestConstants.DEFAULT_CSVTEST_FILE);
+ externalUri = new URI(PublicDatasetTestUtils.DEFAULT_EXTERNAL_FILE);
}
@Test
@@ -74,7 +74,7 @@ public void testSessionTokenDecode() throws Throwable {
= new SessionTokenIdentifier(SESSION_TOKEN_KIND,
alice,
renewer,
- new URI("s3a://landsat-pds/"),
+ new URI("s3a://anything/"),
new MarshalledCredentials("a", "b", ""),
new EncryptionSecrets(S3AEncryptionMethods.SSE_S3, ""),
"origin");
@@ -116,7 +116,7 @@ public void testSessionTokenIdentifierRoundTrip() throws Throwable {
SESSION_TOKEN_KIND,
new Text(),
renewer,
- landsatUri,
+ externalUri,
new MarshalledCredentials("a", "b", "c"),
new EncryptionSecrets(), "");
@@ -135,7 +135,7 @@ public void testSessionTokenIdentifierRoundTripNoRenewer() throws Throwable {
SESSION_TOKEN_KIND,
new Text(),
null,
- landsatUri,
+ externalUri,
new MarshalledCredentials("a", "b", "c"),
new EncryptionSecrets(), "");
@@ -151,7 +151,7 @@ public void testSessionTokenIdentifierRoundTripNoRenewer() throws Throwable {
@Test
public void testRoleTokenIdentifierRoundTrip() throws Throwable {
RoleTokenIdentifier id = new RoleTokenIdentifier(
- landsatUri,
+ externalUri,
new Text(),
new Text(),
new MarshalledCredentials("a", "b", "c"),
@@ -170,7 +170,7 @@ public void testRoleTokenIdentifierRoundTrip() throws Throwable {
public void testFullTokenIdentifierRoundTrip() throws Throwable {
Text renewer = new Text("renewerName");
FullCredentialsTokenIdentifier id = new FullCredentialsTokenIdentifier(
- landsatUri,
+ externalUri,
new Text(),
renewer,
new MarshalledCredentials("a", "b", ""),
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestPaths.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestPaths.java
index ee6480a36af37..e2582cd0a2ee8 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestPaths.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestPaths.java
@@ -26,6 +26,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.test.HadoopTestBase;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.UNIT_TEST_EXAMPLE_PATH;
import static org.apache.hadoop.fs.s3a.commit.staging.Paths.*;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -81,7 +82,7 @@ private void assertUUIDAdded(String path, String expected) {
assertEquals("from " + path, expected, addUUID(path, "UUID"));
}
- private static final String DATA = "s3a://landsat-pds/data/";
+ private static final String DATA = UNIT_TEST_EXAMPLE_PATH;
private static final Path BASE = new Path(DATA);
@Test
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardTool.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardTool.java
index 08696ae62d249..18fdccabaeaf2 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardTool.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardTool.java
@@ -22,14 +22,17 @@
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.InputStreamReader;
+import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.junit.Test;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
import org.apache.hadoop.test.LambdaTestUtils;
import org.apache.hadoop.util.StringUtils;
@@ -40,7 +43,6 @@
import static org.apache.hadoop.fs.s3a.MultipartTestUtils.clearAnyUploads;
import static org.apache.hadoop.fs.s3a.MultipartTestUtils.countUploadsAt;
import static org.apache.hadoop.fs.s3a.MultipartTestUtils.createPartUpload;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVFile;
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.BucketInfo;
import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_BAD_STATE;
@@ -57,36 +59,32 @@ public class ITestS3GuardTool extends AbstractS3GuardToolTestBase {
"-force", "-verbose"};
@Test
- public void testLandsatBucketUnguarded() throws Throwable {
- run(BucketInfo.NAME,
- "-" + BucketInfo.UNGUARDED_FLAG,
- getLandsatCSVFile(getConfiguration()));
- }
-
- @Test
- public void testLandsatBucketRequireGuarded() throws Throwable {
- runToFailure(E_BAD_STATE,
- BucketInfo.NAME,
- "-" + BucketInfo.GUARDED_FLAG,
- getLandsatCSVFile(
- ITestS3GuardTool.this.getConfiguration()));
- }
-
- @Test
- public void testLandsatBucketRequireUnencrypted() throws Throwable {
+ public void testExternalBucketRequireUnencrypted() throws Throwable {
removeBaseAndBucketOverrides(getConfiguration(), S3_ENCRYPTION_ALGORITHM);
run(BucketInfo.NAME,
"-" + BucketInfo.ENCRYPTION_FLAG, "none",
- getLandsatCSVFile(getConfiguration()));
+ externalBucket());
+ }
+
+ /**
+ * Get the external bucket; this is of the default external file.
+ * If not set to the default value, the test will be skipped.
+ * @return the bucket of the default external file.
+ */
+ private String externalBucket() {
+ Configuration conf = getConfiguration();
+ Path result = PublicDatasetTestUtils.requireDefaultExternalData(conf);
+ final URI uri = result.toUri();
+ final String bucket = uri.getScheme() + "://" + uri.getHost();
+ return bucket;
}
@Test
- public void testLandsatBucketRequireEncrypted() throws Throwable {
+ public void testExternalBucketRequireEncrypted() throws Throwable {
runToFailure(E_BAD_STATE,
BucketInfo.NAME,
"-" + BucketInfo.ENCRYPTION_FLAG,
- "AES256", getLandsatCSVFile(
- ITestS3GuardTool.this.getConfiguration()));
+ "AES256", externalBucket());
}
@Test
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestAuthoritativePath.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestAuthoritativePath.java
index c8e56f753bd50..95bb5a567f719 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestAuthoritativePath.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestAuthoritativePath.java
@@ -33,6 +33,7 @@
import org.apache.hadoop.test.AbstractHadoopTestBase;
import static org.apache.hadoop.fs.s3a.Constants.AUTHORITATIVE_PATH;
+import static org.apache.hadoop.fs.s3a.S3ATestConstants.UNIT_TEST_EXAMPLE_PATH;
import static org.assertj.core.api.Assertions.assertThat;
/**
@@ -71,7 +72,7 @@ public void testResolutionWithFQP() throws Throwable {
@Test
public void testOtherBucket() throws Throwable {
assertAuthPaths(l("/one/",
- "s3a://landsat-pds/",
+ UNIT_TEST_EXAMPLE_PATH,
BASE + "/two/"),
"/one/", "/two/");
}
@@ -79,7 +80,7 @@ public void testOtherBucket() throws Throwable {
@Test
public void testOtherScheme() throws Throwable {
assertAuthPaths(l("/one/",
- "s3a://landsat-pds/",
+ UNIT_TEST_EXAMPLE_PATH,
"http://bucket/two/"),
"/one/");
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
index fb9988b29a5c4..ae09452372316 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AInputStreamPerformance.java
@@ -30,6 +30,7 @@
import org.apache.hadoop.fs.s3a.S3AInputStream;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
import org.apache.hadoop.fs.statistics.IOStatistics;
import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
import org.apache.hadoop.fs.statistics.MeanStatistic;
@@ -112,7 +113,9 @@ public void openFS() throws IOException {
Configuration conf = getConf();
conf.setInt(SOCKET_SEND_BUFFER, 16 * 1024);
conf.setInt(SOCKET_RECV_BUFFER, 16 * 1024);
- String testFile = conf.getTrimmed(KEY_CSVTEST_FILE, DEFAULT_CSVTEST_FILE);
+ // look up the test file, no requirement to be set.
+ String testFile = conf.getTrimmed(KEY_CSVTEST_FILE,
+ PublicDatasetTestUtils.DEFAULT_EXTERNAL_FILE);
if (testFile.isEmpty()) {
assumptionMessage = "Empty test property: " + KEY_CSVTEST_FILE;
LOG.warn(assumptionMessage);
@@ -394,6 +397,9 @@ private void executeDecompression(long readahead,
CompressionCodecFactory factory
= new CompressionCodecFactory(getConf());
CompressionCodec codec = factory.getCodec(testData);
+ Assertions.assertThat(codec)
+ .describedAs("No codec found for %s", testData)
+ .isNotNull();
long bytesRead = 0;
int lines = 0;
@@ -525,12 +531,18 @@ private ContractTestUtils.NanoTimer executeRandomIO(S3AInputPolicy policy,
describe("Random IO with policy \"%s\"", policy);
byte[] buffer = new byte[_1MB];
long totalBytesRead = 0;
-
+ final long len = testDataStatus.getLen();
in = openTestFile(policy, 0);
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
for (int[] action : RANDOM_IO_SEQUENCE) {
- int position = action[0];
+ long position = action[0];
int range = action[1];
+ // if a read goes past EOF, fail with details
+ // this will happen if the test datafile is too small.
+ Assertions.assertThat(position + range)
+ .describedAs("readFully(pos=%d range=%d) of %s",
+ position, range, testDataStatus)
+ .isLessThanOrEqualTo(len);
in.readFully(position, buffer, 0, range);
totalBytesRead += range;
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java
index 8c97d896edbde..594cb0cdafb87 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/statistics/ITestAWSStatisticCollection.java
@@ -22,61 +22,30 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
-import org.apache.hadoop.fs.s3a.S3AFileSystem;
-import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest;
-import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_ENDPOINT;
-import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
-import static org.apache.hadoop.fs.s3a.S3ATestUtils.getLandsatCSVPath;
+import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE;
import static org.apache.hadoop.fs.s3a.Statistic.STORE_IO_REQUEST;
-import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
/**
* Verify that AWS SDK statistics are wired up.
- * This test tries to read data from US-east-1 and us-west-2 buckets
- * so as to be confident that the nuances of region mapping
- * are handed correctly (HADOOP-13551).
- * The statistics are probed to verify that the wiring up is complete.
*/
-public class ITestAWSStatisticCollection extends AbstractS3ATestBase {
+public class ITestAWSStatisticCollection extends AbstractS3ACostTest {
- private static final Path COMMON_CRAWL_PATH
- = new Path("s3a://osm-pds/planet/planet-latest.orc");
-
- @Test
- public void testLandsatStatistics() throws Throwable {
- final Configuration conf = getConfiguration();
- // skips the tests if the landsat path isn't the default.
- Path path = getLandsatCSVPath(conf);
- conf.set(ENDPOINT, DEFAULT_ENDPOINT);
- conf.unset("fs.s3a.bucket.landsat-pds.endpoint");
-
- try (S3AFileSystem fs = (S3AFileSystem) path.getFileSystem(conf)) {
- fs.getS3AInternals().getObjectMetadata(path);
- IOStatistics iostats = fs.getIOStatistics();
- assertThatStatisticCounter(iostats,
- STORE_IO_REQUEST.getSymbol())
- .isGreaterThanOrEqualTo(1);
- }
+ @Override
+ public Configuration createConfiguration() {
+ final Configuration conf = super.createConfiguration();
+ conf.setBoolean(FS_S3A_CREATE_PERFORMANCE, true);
+ return conf;
}
@Test
- public void testCommonCrawlStatistics() throws Throwable {
- final Configuration conf = getConfiguration();
- // skips the tests if the landsat path isn't the default.
- getLandsatCSVPath(conf);
-
- Path path = COMMON_CRAWL_PATH;
- conf.set(ENDPOINT, DEFAULT_ENDPOINT);
-
- try (S3AFileSystem fs = (S3AFileSystem) path.getFileSystem(conf)) {
- fs.getS3AInternals().getObjectMetadata(path);
- IOStatistics iostats = fs.getIOStatistics();
- assertThatStatisticCounter(iostats,
- STORE_IO_REQUEST.getSymbol())
- .isGreaterThanOrEqualTo(1);
- }
+ public void testSDKMetricsCostOfGetFileStatusOnFile() throws Throwable {
+ describe("performing getFileStatus on a file");
+ Path simpleFile = file(methodPath());
+ // and repeat on the file looking at AWS wired up stats
+ verifyMetrics(() -> getFileSystem().getFileStatus(simpleFile),
+ with(STORE_IO_REQUEST, 1));
}
}
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/PublicDatasetTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/PublicDatasetTestUtils.java
index 7ef2449b8e83f..3835548b1e251 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/PublicDatasetTestUtils.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/test/PublicDatasetTestUtils.java
@@ -18,9 +18,13 @@
package org.apache.hadoop.fs.s3a.test;
+import org.junit.Assume;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.S3ATestConstants;
import org.apache.hadoop.fs.s3a.S3ATestUtils;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.KEY_BUCKET_WITH_MANY_OBJECTS;
@@ -69,6 +73,77 @@ private PublicDatasetTestUtils() {}
private static final String DEFAULT_BUCKET_WITH_MANY_OBJECTS
= "s3a://usgs-landsat/collection02/level-1/";
+ /**
+ * ORC dataset: {@value}.
+ */
+ private static final Path ORC_DATA = new Path("s3a://osm-pds/planet/planet-latest.orc");
+
+ /**
+ * Provide a Path for some ORC data.
+ *
+ * @param conf Hadoop configuration
+ * @return S3A FS URI
+ */
+ public static Path getOrcData(Configuration conf) {
+ return ORC_DATA;
+ }
+
+ /**
+ * Default path for the external test file: {@value}.
+ * This must be: gzipped, large enough for the performance
+ * tests and in a read-only bucket with anonymous access.
+ * */
+ public static final String DEFAULT_EXTERNAL_FILE =
+ "s3a://noaa-cors-pds/raw/2023/017/ohfh/OHFH017d.23_.gz";
+
+ /**
+ * Get the external test file.
+ *
+ * This must be: gzipped, large enough for the performance
+ * tests and in a read-only bucket with anon
+ * @param conf configuration
+ * @return a dataset which meets the requirements.
+ */
+ public static Path getExternalData(Configuration conf) {
+ return new Path(fetchFromConfig(conf,
+ S3ATestConstants.KEY_CSVTEST_FILE, DEFAULT_EXTERNAL_FILE));
+ }
+
+ /**
+ * Get the anonymous dataset..
+ * @param conf configuration
+ * @return a dataset which supports anonymous access.
+ */
+ public static Path requireAnonymousDataPath(Configuration conf) {
+ return requireDefaultExternalData(conf);
+ }
+
+
+ /**
+ * Get the external test file; assume() that it is not modified (i.e. we haven't
+ * switched to a new storage infrastructure where the bucket is no longer
+ * read only).
+ * @return test file.
+ * @param conf test configuration
+ */
+ public static String requireDefaultExternalDataFile(Configuration conf) {
+ String filename = getExternalData(conf).toUri().toString();
+ Assume.assumeTrue("External test file is not the default",
+ DEFAULT_EXTERNAL_FILE.equals(filename));
+ return filename;
+ }
+
+ /**
+ * Get the test external file; assume() that it is not modified (i.e. we haven't
+ * switched to a new storage infrastructure where the bucket is no longer
+ * read only).
+ * @param conf test configuration
+ * @return test file as a path.
+ */
+ public static Path requireDefaultExternalData(Configuration conf) {
+ return new Path(requireDefaultExternalDataFile(conf));
+ }
+
/**
* Provide a URI for a directory containing many objects.
*
@@ -97,6 +172,13 @@ public static String getRequesterPaysObject(Configuration conf) {
KEY_REQUESTER_PAYS_FILE, DEFAULT_REQUESTER_PAYS_FILE);
}
+ /**
+ * Fetch a trimmed configuration value, require it to to be non-empty.
+ * @param conf configuration file
+ * @param key key
+ * @param defaultValue default value.
+ * @return the resolved value.
+ */
private static String fetchFromConfig(Configuration conf, String key, String defaultValue) {
String value = conf.getTrimmed(key, defaultValue);
diff --git a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
index 30e54ca0b87db..1abec3e0706f1 100644
--- a/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
+++ b/hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
@@ -30,37 +30,57 @@
false
-
+
- fs.s3a.bucket.landsat-pds.multipart.purge
+ fs.s3a.bucket.noaa-cors-pds.endpoint.region
+ us-east-1
+
+
+
+ fs.s3a.bucket.noaa-isd-pds.multipart.purge
false
Don't try to purge uploads in the read-only bucket, as
it will only create log noise.
- fs.s3a.bucket.landsat-pds.probe
+ fs.s3a.bucket.noaa-isd-pds.probe
0
Let's postpone existence checks to the first IO operation
- fs.s3a.bucket.landsat-pds.audit.add.referrer.header
+ fs.s3a.bucket.noaa-isd-pds.audit.add.referrer.header
false
- Do not add the referrer header to landsat operations
+ Do not add the referrer header
+
+
+
+ fs.s3a.bucket.noaa-isd-pds.prefetch.block.size
+ 128k
+ Use a small prefetch size so tests fetch multiple blocks