Revert "HADOOP-18168. . (#4140)"

This reverts commit 6ab7b72cd6.
This commit is contained in:
Steve Loughran 2022-05-03 14:27:00 +01:00 committed by GitHub
parent 6ab7b72cd6
commit ed6b0e4bda
6 changed files with 18 additions and 170 deletions

View File

@ -544,18 +544,6 @@ 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

View File

@ -26,11 +26,11 @@ import org.apache.hadoop.conf.Configuration;
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;
@ -42,15 +42,10 @@ public class ITestS3ARequesterPays extends AbstractS3ATestBase {
@Override
protected Configuration createConfiguration() {
Configuration conf = super.createConfiguration();
Path requesterPaysPath = getRequesterPaysPath(conf);
String requesterPaysBucketName = requesterPaysPath.toUri().getHost();
S3ATestUtils.removeBaseAndBucketOverrides(
requesterPaysBucketName,
conf,
S3ATestUtils.removeBaseAndBucketOverrides(conf,
ALLOW_REQUESTER_PAYS,
ENDPOINT,
S3A_BUCKET_PROBE);
return conf;
}
@ -107,8 +102,14 @@ public class ITestS3ARequesterPays extends AbstractS3ATestBase {
}
}
private static Path getRequesterPaysPath(Configuration conf) {
return new Path(PublicDatasetTestUtils.getRequesterPaysObject(conf));
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);
}
}

View File

@ -20,9 +20,6 @@ package org.apache.hadoop.fs.s3a;
import java.time.Duration;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
/**
* Constants for S3A Testing.
*/
@ -102,19 +99,14 @@ public interface S3ATestConstants {
/**
* Configuration key for an existing object in a requester pays bucket: {@value}.
*
* Accessible via
* {@link PublicDatasetTestUtils#getRequesterPaysObject(Configuration)}.
* If not set, defaults to {@value DEFAULT_REQUESTER_PAYS_FILE}.
*/
String KEY_REQUESTER_PAYS_FILE = TEST_FS_S3A + "requester.pays.file";
/**
* Configuration key for an existing bucket with many objects: {@value}.
*
* This is used for tests depending on buckets with a large number of keys.
* Default path for an S3 object inside a requester pays enabled bucket: {@value}.
*/
String KEY_BUCKET_WITH_MANY_OBJECTS
= TEST_FS_S3A + "bucket-with-many-objects";
String DEFAULT_REQUESTER_PAYS_FILE = "s3a://usgs-landsat/collection02/catalog.json";
/**
* Name of the property to define the timeout for scale tests: {@value}.

View File

@ -1,101 +0,0 @@
/*
* 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;
}
}

View File

@ -28,11 +28,9 @@ import org.junit.Test;
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;
@ -309,25 +307,22 @@ public class ITestMarkerTool extends AbstractMarkerToolTest {
}
/**
* Run an audit against a bucket with a large number of objects.
* Run an audit against the landsat bucket.
* <p></p>
* This tests paging/scale against a larger bucket without
* worrying about setup costs.
*/
@Test
public void testRunAuditManyObjectsInBucket() throws Throwable {
describe("Audit a few thousand objects");
public void testRunLimitedLandsatAudit() throws Throwable {
describe("Audit a few thousand landsat 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,
bucketUri);
LANDSAT_BUCKET);
readOutput(audit);
}

View File

@ -30,8 +30,6 @@
<final>false</final>
</property>
<!-- Per-bucket configurations: landsat-pds -->
<property>
<name>fs.s3a.bucket.landsat-pds.endpoint</name>
<value>${central.endpoint}</value>
@ -57,31 +55,6 @@
<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