From fa918915cad88f64283b4ccc276caf5fe5b7aea6 Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Wed, 7 Feb 2024 23:15:41 -0800 Subject: [PATCH] HADOOP-19066. S3A: AWS SDK V2 - Enabling FIPS should be allowed with central endpoint --- .../hadoop/fs/s3a/DefaultS3ClientFactory.java | 7 +- .../markdown/tools/hadoop-aws/connecting.md | 15 ++- .../hadoop/fs/s3a/ITestS3AEndpointRegion.java | 112 +++++++++++++++--- 3 files changed, 114 insertions(+), 20 deletions(-) diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java index 284ba8e6ae5c9..7f6978e8e9284 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java @@ -110,7 +110,7 @@ public class DefaultS3ClientFactory extends Configured */ @VisibleForTesting public static final String ERROR_ENDPOINT_WITH_FIPS = - "An endpoint cannot set when " + FIPS_ENDPOINT + " is true"; + "Non central endpoint cannot be set when " + FIPS_ENDPOINT + " is true"; @Override public S3Client createS3Client( @@ -290,10 +290,11 @@ private , ClientT> void builder.fipsEnabled(fipsEnabled); if (endpoint != null) { - checkArgument(!fipsEnabled, - "%s : %s", ERROR_ENDPOINT_WITH_FIPS, endpoint); boolean endpointEndsWithCentral = endpointStr.endsWith(CENTRAL_ENDPOINT); + checkArgument(!fipsEnabled || endpointEndsWithCentral, "%s : %s", + ERROR_ENDPOINT_WITH_FIPS, + endpoint); // No region was configured, // determine the region from the endpoint. diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/connecting.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/connecting.md index 51e70ef231bf7..374f455cfd54a 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/connecting.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/connecting.md @@ -360,8 +360,18 @@ For a single bucket: ``` -If this option is `true`, the endpoint option `fs.s3a.endpoint` MUST NOT be set: +If `fs.s3a.endpoint.fips` is `true`, the endpoint option `fs.s3a.endpoint` MUST NOT be set to +any non-central endpoint value. If `fs.s3a.endpoint.fips` is `true`, the only *optionally* allowed +value for `fs.s3a.endpoint` is central endpoint `s3.amazonaws.com`. +S3A error message if `s3.eu-west-2.amazonaws.com` endpoint is used with FIPS: +``` +Non central endpoint cannot be set when fs.s3a.endpoint.fips is true : https://s3.eu-west-2.amazonaws.com +``` + +S3A validation is used to fail-fast before the SDK returns error. + +AWS SDK error message if S3A does not fail-fast: ``` A custom endpoint cannot be combined with FIPS: https://s3.eu-west-2.amazonaws.com ``` @@ -379,6 +389,9 @@ Received an UnknownHostException when attempting to interact with a service. ``` +For more details on endpoint and region settings, please check +[S3 endpoint and region settings in detail](connecting.html#s3_endpoint_region_details). + *Important* OpenSSL and FIPS endpoints Linux distributions with an FIPS-compliant SSL library may not be compatible with wildfly. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java index 95f31d7527f86..8fbdcf0b3466f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java @@ -48,6 +48,7 @@ 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.FIPS_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; @@ -156,6 +157,11 @@ public void testCentralEndpoint() throws Throwable { S3Client client = createS3Client(conf, CENTRAL_ENDPOINT, null, US_EAST_2, false); expectInterceptorException(client); + + client = createS3Client(conf, CENTRAL_ENDPOINT, null, + US_EAST_2, true); + + expectInterceptorException(client); } @Test @@ -168,11 +174,21 @@ public void testCentralEndpointWithRegion() throws Throwable { expectInterceptorException(client); + client = createS3Client(conf, CENTRAL_ENDPOINT, US_WEST_2, + US_WEST_2, true); + + expectInterceptorException(client); + client = createS3Client(conf, CENTRAL_ENDPOINT, US_EAST_1, US_EAST_1, false); expectInterceptorException(client); + client = createS3Client(conf, CENTRAL_ENDPOINT, US_EAST_1, + US_EAST_1, true); + + expectInterceptorException(client); + } @Test @@ -203,7 +219,7 @@ public void testWithFipsAndEndpoint() throws Throwable { describe("Create a client with fips and an endpoint"); intercept(IllegalArgumentException.class, ERROR_ENDPOINT_WITH_FIPS, () -> - createS3Client(getConfiguration(), CENTRAL_ENDPOINT, null, US_EAST_1, true)); + createS3Client(getConfiguration(), US_WEST_2, null, US_EAST_1, true)); } @Test @@ -293,7 +309,8 @@ public void testCentralEndpointAndDifferentRegionThanBucket() throws Throwable { ENDPOINT, AWS_REGION, ALLOW_REQUESTER_PAYS, - KEY_REQUESTER_PAYS_FILE); + KEY_REQUESTER_PAYS_FILE, + FIPS_ENDPOINT); removeBaseAndBucketOverrides( DEFAULT_REQUESTER_PAYS_BUCKET_NAME, @@ -301,20 +318,14 @@ public void testCentralEndpointAndDifferentRegionThanBucket() throws Throwable { ENDPOINT, AWS_REGION, ALLOW_REQUESTER_PAYS, - KEY_REQUESTER_PAYS_FILE); + KEY_REQUESTER_PAYS_FILE, + FIPS_ENDPOINT); 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(); + assertRequesterPaysFileExistence(newConf); } @Test @@ -329,7 +340,8 @@ public void testCentralEndpointAndSameRegionAsBucket() throws Throwable { ENDPOINT, AWS_REGION, ALLOW_REQUESTER_PAYS, - KEY_REQUESTER_PAYS_FILE); + KEY_REQUESTER_PAYS_FILE, + FIPS_ENDPOINT); removeBaseAndBucketOverrides( DEFAULT_REQUESTER_PAYS_BUCKET_NAME, @@ -337,15 +349,59 @@ public void testCentralEndpointAndSameRegionAsBucket() throws Throwable { ENDPOINT, AWS_REGION, ALLOW_REQUESTER_PAYS, - KEY_REQUESTER_PAYS_FILE); + KEY_REQUESTER_PAYS_FILE, + FIPS_ENDPOINT); newConf.set(ENDPOINT, CENTRAL_ENDPOINT); newConf.set(AWS_REGION, US_WEST_2); newConf.setBoolean(ALLOW_REQUESTER_PAYS, true); + assertRequesterPaysFileExistence(newConf); + } + + @Test + public void testCentralEndpointAndFipsForPublicBucket() throws Throwable { + describe("Access public bucket using central endpoint and region " + + "same as that of the public bucket with fips enabled"); + final Configuration conf = getConfiguration(); + final Configuration newConf = new Configuration(conf); + + removeBaseAndBucketOverrides( + newConf, + ENDPOINT, + AWS_REGION, + ALLOW_REQUESTER_PAYS, + KEY_REQUESTER_PAYS_FILE, + FIPS_ENDPOINT); + + removeBaseAndBucketOverrides( + DEFAULT_REQUESTER_PAYS_BUCKET_NAME, + newConf, + ENDPOINT, + AWS_REGION, + ALLOW_REQUESTER_PAYS, + KEY_REQUESTER_PAYS_FILE, + FIPS_ENDPOINT); + + newConf.set(ENDPOINT, CENTRAL_ENDPOINT); + newConf.set(AWS_REGION, US_WEST_2); + newConf.setBoolean(ALLOW_REQUESTER_PAYS, true); + newConf.setBoolean(FIPS_ENDPOINT, true); + + assertRequesterPaysFileExistence(newConf); + } + + /** + * Assert that the file exists on the requester pays public bucket. + * + * @param conf the configuration object. + * @throws IOException if file system operations encounter errors. + */ + private void assertRequesterPaysFileExistence(Configuration conf) + throws IOException { Path filePath = new Path(PublicDatasetTestUtils - .getRequesterPaysObject(newConf)); - newFS = (S3AFileSystem) filePath.getFileSystem(newConf); + .getRequesterPaysObject(conf)); + newFS = (S3AFileSystem) filePath.getFileSystem(conf); Assertions .assertThat(newFS.exists(filePath)) @@ -364,9 +420,33 @@ public void testCentralEndpointAndNullRegionWithCRUD() throws Throwable { removeBaseAndBucketOverrides( newConf, ENDPOINT, - AWS_REGION); + AWS_REGION, + FIPS_ENDPOINT); + + newConf.set(ENDPOINT, CENTRAL_ENDPOINT); + + newFS = new S3AFileSystem(); + newFS.initialize(getFileSystem().getUri(), newConf); + + assertOpsUsingNewFs(); + } + + @Test + public void testCentralEndpointAndNullRegionFipsWithCRUD() throws Throwable { + describe("Access the test bucket using central endpoint and" + + " null region and fips enabled, perform file system CRUD operations"); + final Configuration conf = getConfiguration(); + + final Configuration newConf = new Configuration(conf); + + removeBaseAndBucketOverrides( + newConf, + ENDPOINT, + AWS_REGION, + FIPS_ENDPOINT); newConf.set(ENDPOINT, CENTRAL_ENDPOINT); + newConf.setBoolean(FIPS_ENDPOINT, true); newFS = new S3AFileSystem(); newFS.initialize(getFileSystem().getUri(), newConf);