|
@@ -38,13 +38,20 @@ import software.amazon.awssdk.services.s3.model.HeadBucketRequest;
|
|
|
import software.amazon.awssdk.services.s3.model.HeadBucketResponse;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext;
|
|
|
+import org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils;
|
|
|
|
|
|
+import static org.apache.hadoop.fs.s3a.Constants.ALLOW_REQUESTER_PAYS;
|
|
|
import static org.apache.hadoop.fs.s3a.Constants.AWS_REGION;
|
|
|
import static org.apache.hadoop.fs.s3a.Constants.CENTRAL_ENDPOINT;
|
|
|
+import static org.apache.hadoop.fs.s3a.Constants.ENDPOINT;
|
|
|
import static org.apache.hadoop.fs.s3a.Constants.PATH_STYLE_ACCESS;
|
|
|
import static org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.ERROR_ENDPOINT_WITH_FIPS;
|
|
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
|
|
|
+import static org.apache.hadoop.fs.s3a.test.PublicDatasetTestUtils.DEFAULT_REQUESTER_PAYS_BUCKET_NAME;
|
|
|
import static org.apache.hadoop.io.IOUtils.closeStream;
|
|
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
|
|
|
@@ -146,11 +153,28 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
|
|
|
describe("Create a client with the central endpoint");
|
|
|
Configuration conf = getConfiguration();
|
|
|
|
|
|
- S3Client client = createS3Client(conf, CENTRAL_ENDPOINT, null, US_EAST_1, false);
|
|
|
+ S3Client client = createS3Client(conf, CENTRAL_ENDPOINT, null, US_EAST_2, false);
|
|
|
|
|
|
expectInterceptorException(client);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testCentralEndpointWithRegion() throws Throwable {
|
|
|
+ describe("Create a client with the central endpoint but also specify region");
|
|
|
+ Configuration conf = getConfiguration();
|
|
|
+
|
|
|
+ S3Client client = createS3Client(conf, CENTRAL_ENDPOINT, US_WEST_2,
|
|
|
+ US_WEST_2, false);
|
|
|
+
|
|
|
+ expectInterceptorException(client);
|
|
|
+
|
|
|
+ client = createS3Client(conf, CENTRAL_ENDPOINT, US_EAST_1,
|
|
|
+ US_EAST_1, false);
|
|
|
+
|
|
|
+ expectInterceptorException(client);
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testWithRegionConfig() throws Throwable {
|
|
|
describe("Create a client with a configured region");
|
|
@@ -257,6 +281,141 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
|
|
|
expectInterceptorException(client);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testCentralEndpointAndDifferentRegionThanBucket() throws Throwable {
|
|
|
+ describe("Access public bucket using central endpoint and region "
|
|
|
+ + "different than that of the public bucket");
|
|
|
+ final Configuration conf = getConfiguration();
|
|
|
+ final Configuration newConf = new Configuration(conf);
|
|
|
+
|
|
|
+ removeBaseAndBucketOverrides(
|
|
|
+ newConf,
|
|
|
+ ENDPOINT,
|
|
|
+ AWS_REGION,
|
|
|
+ ALLOW_REQUESTER_PAYS,
|
|
|
+ KEY_REQUESTER_PAYS_FILE);
|
|
|
+
|
|
|
+ removeBaseAndBucketOverrides(
|
|
|
+ DEFAULT_REQUESTER_PAYS_BUCKET_NAME,
|
|
|
+ newConf,
|
|
|
+ ENDPOINT,
|
|
|
+ AWS_REGION,
|
|
|
+ ALLOW_REQUESTER_PAYS,
|
|
|
+ KEY_REQUESTER_PAYS_FILE);
|
|
|
+
|
|
|
+ newConf.set(ENDPOINT, CENTRAL_ENDPOINT);
|
|
|
+ newConf.set(AWS_REGION, EU_WEST_1);
|
|
|
+ newConf.setBoolean(ALLOW_REQUESTER_PAYS, true);
|
|
|
+
|
|
|
+ Path filePath = new Path(PublicDatasetTestUtils
|
|
|
+ .getRequesterPaysObject(newConf));
|
|
|
+ newFS = (S3AFileSystem) filePath.getFileSystem(newConf);
|
|
|
+
|
|
|
+ Assertions
|
|
|
+ .assertThat(newFS.exists(filePath))
|
|
|
+ .describedAs("Existence of path: " + filePath)
|
|
|
+ .isTrue();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCentralEndpointAndSameRegionAsBucket() throws Throwable {
|
|
|
+ describe("Access public bucket using central endpoint and region "
|
|
|
+ + "same as that of the public bucket");
|
|
|
+ final Configuration conf = getConfiguration();
|
|
|
+ final Configuration newConf = new Configuration(conf);
|
|
|
+
|
|
|
+ removeBaseAndBucketOverrides(
|
|
|
+ newConf,
|
|
|
+ ENDPOINT,
|
|
|
+ AWS_REGION,
|
|
|
+ ALLOW_REQUESTER_PAYS,
|
|
|
+ KEY_REQUESTER_PAYS_FILE);
|
|
|
+
|
|
|
+ removeBaseAndBucketOverrides(
|
|
|
+ DEFAULT_REQUESTER_PAYS_BUCKET_NAME,
|
|
|
+ newConf,
|
|
|
+ ENDPOINT,
|
|
|
+ AWS_REGION,
|
|
|
+ ALLOW_REQUESTER_PAYS,
|
|
|
+ KEY_REQUESTER_PAYS_FILE);
|
|
|
+
|
|
|
+ newConf.set(ENDPOINT, CENTRAL_ENDPOINT);
|
|
|
+ newConf.set(AWS_REGION, US_WEST_2);
|
|
|
+ newConf.setBoolean(ALLOW_REQUESTER_PAYS, true);
|
|
|
+
|
|
|
+ Path filePath = new Path(PublicDatasetTestUtils
|
|
|
+ .getRequesterPaysObject(newConf));
|
|
|
+ newFS = (S3AFileSystem) filePath.getFileSystem(newConf);
|
|
|
+
|
|
|
+ Assertions
|
|
|
+ .assertThat(newFS.exists(filePath))
|
|
|
+ .describedAs("Existence of path: " + filePath)
|
|
|
+ .isTrue();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testCentralEndpointAndNullRegionWithCRUD() throws Throwable {
|
|
|
+ describe("Access the test bucket using central endpoint and"
|
|
|
+ + " null region, perform file system CRUD operations");
|
|
|
+ final Configuration conf = getConfiguration();
|
|
|
+
|
|
|
+ final Configuration newConf = new Configuration(conf);
|
|
|
+
|
|
|
+ removeBaseAndBucketOverrides(
|
|
|
+ newConf,
|
|
|
+ ENDPOINT,
|
|
|
+ AWS_REGION);
|
|
|
+
|
|
|
+ newConf.set(ENDPOINT, CENTRAL_ENDPOINT);
|
|
|
+
|
|
|
+ newFS = new S3AFileSystem();
|
|
|
+ newFS.initialize(getFileSystem().getUri(), newConf);
|
|
|
+
|
|
|
+ assertOpsUsingNewFs();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void assertOpsUsingNewFs() throws IOException {
|
|
|
+ final String file = getMethodName();
|
|
|
+ final Path basePath = methodPath();
|
|
|
+ final Path srcDir = new Path(basePath, "srcdir");
|
|
|
+ newFS.mkdirs(srcDir);
|
|
|
+ Path srcFilePath = new Path(srcDir, file);
|
|
|
+
|
|
|
+ try (FSDataOutputStream out = newFS.create(srcFilePath)) {
|
|
|
+ out.write(new byte[] {1, 2, 3});
|
|
|
+ }
|
|
|
+
|
|
|
+ Assertions
|
|
|
+ .assertThat(newFS.exists(srcFilePath))
|
|
|
+ .describedAs("Existence of file: " + srcFilePath)
|
|
|
+ .isTrue();
|
|
|
+ Assertions
|
|
|
+ .assertThat(getFileSystem().exists(srcFilePath))
|
|
|
+ .describedAs("Existence of file: " + srcFilePath)
|
|
|
+ .isTrue();
|
|
|
+
|
|
|
+ byte[] buffer = new byte[3];
|
|
|
+
|
|
|
+ try (FSDataInputStream in = newFS.open(srcFilePath)) {
|
|
|
+ in.readFully(buffer);
|
|
|
+ Assertions
|
|
|
+ .assertThat(buffer)
|
|
|
+ .describedAs("Contents read from " + srcFilePath)
|
|
|
+ .containsExactly(1, 2, 3);
|
|
|
+ }
|
|
|
+
|
|
|
+ newFS.delete(srcDir, true);
|
|
|
+
|
|
|
+ Assertions
|
|
|
+ .assertThat(newFS.exists(srcFilePath))
|
|
|
+ .describedAs("Existence of file: " + srcFilePath + " using new FS")
|
|
|
+ .isFalse();
|
|
|
+ Assertions
|
|
|
+ .assertThat(getFileSystem().exists(srcFilePath))
|
|
|
+ .describedAs("Existence of file: " + srcFilePath + " using original FS")
|
|
|
+ .isFalse();
|
|
|
+ }
|
|
|
+
|
|
|
private final class RegionInterceptor implements ExecutionInterceptor {
|
|
|
private final String endpoint;
|
|
|
private final String region;
|
|
@@ -272,7 +431,7 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
|
|
|
public void beforeExecution(Context.BeforeExecution context,
|
|
|
ExecutionAttributes executionAttributes) {
|
|
|
|
|
|
- if (endpoint != null) {
|
|
|
+ if (endpoint != null && !endpoint.endsWith(CENTRAL_ENDPOINT)) {
|
|
|
Assertions.assertThat(
|
|
|
executionAttributes.getAttribute(AwsExecutionAttribute.ENDPOINT_OVERRIDDEN))
|
|
|
.describedAs("Endpoint not overridden").isTrue();
|