Skip to content

HADOOP-18168. Fix S3A ITestMarkerTool dep. on purged public bucket, introduce PublicDatasetTestUtils #4140

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 19 commits into from
May 3, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
19 commits
Select commit Hold shift + click to select a range
cdbb84c
Update ITestMarkerTool to take many-object bucket from config
dannycjones Apr 5, 2022
0a06a92
Replace default test many-object bucket with 'commoncrawl'
dannycjones Apr 5, 2022
8152d19
Rename testRunLimitedLandsatAudit test
dannycjones Apr 5, 2022
83d62c1
Introduce PublicDatasetTestUtils for bucket with many objects
dannycjones Apr 26, 2022
f058fe4
Migrate ITestS3ARequesterPays to PublicDatasetTestUtils
dannycjones Apr 26, 2022
f73cb3f
Replace common-crawl usage with requester pays bucket usgs-landsat
dannycjones Apr 26, 2022
0c39b59
Add method for large test object (landsat) to PublicDatasetTestUtils
dannycjones Apr 26, 2022
a8e56f3
Revert "Add method for large test object (landsat) to PublicDatasetTe…
dannycjones Apr 27, 2022
8fd79b5
Add private constructor for PublicDatasetTestUtils
dannycjones Apr 27, 2022
11b9541
Fix bucket override configurations for ITestS3ARequesterPays
dannycjones Apr 27, 2022
e734caa
Merge branch 'trunk' into HADOOP-18168
dannycjones Apr 27, 2022
79710a1
Allow endpoint to be configured using bucket overrides
dannycjones Apr 27, 2022
1692c71
Mark PublicDatasetTestUtils class as final
dannycjones Apr 27, 2022
364ce9e
Fix ITestS3ARequesterPays#updateConf JavaDoc
dannycjones Apr 28, 2022
73eeb50
Revert moving super.createConfiguration() logic into helper method
dannycjones Apr 28, 2022
7bbc2b2
Fix S3ATestConstants import
dannycjones Apr 29, 2022
cb19dff
Move PublicDatasetTestUtils to o.a.h.fs.s3a.test
dannycjones Apr 29, 2022
a736bb5
Add note on PublicDatasetTestUtils to testing.md
dannycjones Apr 29, 2022
6302c5b
Fix note 'standard commercial partition'
dannycjones Apr 29, 2022
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -544,6 +544,18 @@ which address issues. In particular, we encourage testing of Hadoop release
candidates, as these third-party endpoints get even less testing than the
S3 endpoint itself.

### Public datasets used in tests

Some tests rely on the presence of existing public datasets available on Amazon S3.
You may find a number of these in `org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils`.

When testing against an endpoint which is not part of Amazon S3's standard commercial partition
(`aws`) such as third-party implementations or AWS's China regions, you should replace these
configurations with an empty space (` `) to disable the tests or an existing path in your object
store that supports these tests.

An example of this might be the MarkerTools tests which require a bucket with a large number of
objects or the requester pays tests that require requester pays to be enabled for the bucket.

### Disabling the encryption tests

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,11 @@
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
import org.apache.hadoop.fs.statistics.StreamStatisticNames;

import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS;
import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
import static org.apache.hadoop.fs.s3a.Constants.S3A_BUCKET_PROBE;
import static org.apache.hadoop.test.LambdaTestUtils.intercept;

Expand All @@ -42,10 +42,15 @@ public class ITestS3ARequesterPays extends AbstractS3ATestBase {
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
S3ATestUtils.removeBaseAndBucketOverrides(conf,

Path requesterPaysPath = getRequesterPaysPath(conf);
String requesterPaysBucketName = requesterPaysPath.toUri().getHost();
S3ATestUtils.removeBaseAndBucketOverrides(
requesterPaysBucketName,
conf,
ALLOW_REQUESTER_PAYS,
ENDPOINT,
S3A_BUCKET_PROBE);

return conf;
}

Expand Down Expand Up @@ -102,14 +107,8 @@ public void testRequesterPaysDisabledFails() throws Throwable {
}
}

private Path getRequesterPaysPath(Configuration conf) {
String requesterPaysFile =
conf.getTrimmed(KEY_REQUESTER_PAYS_FILE, DEFAULT_REQUESTER_PAYS_FILE);
S3ATestUtils.assume(
"Empty test property: " + KEY_REQUESTER_PAYS_FILE,
!requesterPaysFile.isEmpty()
);
return new Path(requesterPaysFile);
private static Path getRequesterPaysPath(Configuration conf) {
return new Path(PublicDatasetTestUtils.getRequesterPaysObject(conf));
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,9 @@

import java.time.Duration;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;

/**
* Constants for S3A Testing.
*/
Expand Down Expand Up @@ -99,14 +102,19 @@ public interface S3ATestConstants {

/**
* Configuration key for an existing object in a requester pays bucket: {@value}.
* If not set, defaults to {@value DEFAULT_REQUESTER_PAYS_FILE}.
*
* Accessible via
* {@link PublicDatasetTestUtils#getRequesterPaysObject(Configuration)}.
*/
String KEY_REQUESTER_PAYS_FILE = TEST_FS_S3A + "requester.pays.file";

/**
* Default path for an S3 object inside a requester pays enabled bucket: {@value}.
* Configuration key for an existing bucket with many objects: {@value}.
*
* This is used for tests depending on buckets with a large number of keys.
*/
String DEFAULT_REQUESTER_PAYS_FILE = "s3a://usgs-landsat/collection02/catalog.json";
String KEY_BUCKET_WITH_MANY_OBJECTS
= TEST_FS_S3A + "bucket-with-many-objects";

/**
* Name of the property to define the timeout for scale tests: {@value}.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,101 @@
/*
* 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.s3a.test;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.S3ATestUtils;

import static org.apache.hadoop.fs.s3a.S3ATestConstants.KEY_BUCKET_WITH_MANY_OBJECTS;
import static org.apache.hadoop.fs.s3a.S3ATestConstants.KEY_REQUESTER_PAYS_FILE;

/**
* Provides S3A filesystem URIs for public data sets for specific use cases.
*
* This allows for the contract between S3A tests and the existence of data sets
* to be explicit and also standardizes access and configuration of
* replacements.
*
* Bucket specific configuration such as endpoint or requester pays should be
* configured within "hadoop-tools/hadoop-aws/src/test/resources/core-site.xml".
*
* Warning: methods may mutate the configuration instance passed in.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public final class PublicDatasetTestUtils {

/**
* Private constructor for utility class.
*/
private PublicDatasetTestUtils() {}

/**
* Default path for an object inside a requester pays bucket: {@value}.
*/
private static final String DEFAULT_REQUESTER_PAYS_FILE
= "s3a://usgs-landsat/collection02/catalog.json";

/**
* Default bucket for an S3A file system with many objects: {@value}.
*
* We use a subdirectory to ensure we have permissions on all objects
* contained within as well as permission to inspect the directory itself.
*/
private static final String DEFAULT_BUCKET_WITH_MANY_OBJECTS
= "s3a://usgs-landsat/collection02/level-1/";

/**
* Provide a URI for a directory containing many objects.
*
* Unless otherwise configured,
* this will be {@value DEFAULT_BUCKET_WITH_MANY_OBJECTS}.
*
* @param conf Hadoop configuration
* @return S3A FS URI
*/
public static String getBucketPrefixWithManyObjects(Configuration conf) {
return fetchFromConfig(conf,
KEY_BUCKET_WITH_MANY_OBJECTS, DEFAULT_BUCKET_WITH_MANY_OBJECTS);
}

/**
* Provide a URI to an object within a requester pays enabled bucket.
*
* Unless otherwise configured,
* this will be {@value DEFAULT_REQUESTER_PAYS_FILE}.
*
* @param conf Hadoop configuration
* @return S3A FS URI
*/
public static String getRequesterPaysObject(Configuration conf) {
return fetchFromConfig(conf,
KEY_REQUESTER_PAYS_FILE, DEFAULT_REQUESTER_PAYS_FILE);
}

private static String fetchFromConfig(Configuration conf, String key, String defaultValue) {
String value = conf.getTrimmed(key, defaultValue);

S3ATestUtils.assume("Empty test property: " + key, !value.isEmpty());

return value;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,11 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.contract.ContractTestUtils;
import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
import org.apache.hadoop.fs.s3a.S3AFileSystem;

import static org.apache.hadoop.fs.s3a.Constants.DIRECTORY_MARKER_POLICY_AUTHORITATIVE;
Expand Down Expand Up @@ -307,22 +309,25 @@ public void testRunLimitedAudit() throws Throwable {
}

/**
* Run an audit against the landsat bucket.
* Run an audit against a bucket with a large number of objects.
* <p></p>
* This tests paging/scale against a larger bucket without
* worrying about setup costs.
*/
@Test
public void testRunLimitedLandsatAudit() throws Throwable {
describe("Audit a few thousand landsat objects");
public void testRunAuditManyObjectsInBucket() throws Throwable {
describe("Audit a few thousand objects");
final File audit = tempAuditFile();

Configuration conf = super.createConfiguration();
String bucketUri = PublicDatasetTestUtils.getBucketPrefixWithManyObjects(conf);

runToFailure(EXIT_INTERRUPTED,
MARKERS,
AUDIT,
m(OPT_LIMIT), 3000,
m(OPT_OUT), audit,
LANDSAT_BUCKET);
bucketUri);
readOutput(audit);
}

Expand Down
27 changes: 27 additions & 0 deletions hadoop-tools/hadoop-aws/src/test/resources/core-site.xml
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
<final>false</final>
</property>

<!-- Per-bucket configurations: landsat-pds -->

<property>
<name>fs.s3a.bucket.landsat-pds.endpoint</name>
<value>${central.endpoint}</value>
Expand All @@ -55,6 +57,31 @@
<description>Do not add the referrer header to landsat operations</description>
</property>

<!-- Per-bucket configurations: usgs-landsat -->

<property>
<name>fs.s3a.bucket.usgs-landsat.endpoint</name>
<value>${central.endpoint}</value>
</property>

<property>
<name>fs.s3a.bucket.usgs-landsat.requester.pays.enabled</name>
<value>true</value>
<description>usgs-landsat requires requester pays enabled</description>
</property>

<property>
<name>fs.s3a.bucket.usgs-landsat.multipart.purge</name>
<value>false</value>
<description>Don't try to purge uploads in the read-only bucket, as
it will only create log noise.</description>
</property>

<property>
<name>fs.s3a.bucket.usgs-landsat.audit.add.referrer.header</name>
<value>false</value>
</property>


<!--
This is the default endpoint, which can be used to interact
Expand Down