Selaa lähdekoodia

HADOOP-19066. S3A: AWS SDK V2 - Enabling FIPS should be allowed with central endpoint (#6539)

Includes HADOOP-19066. Run FIPS test for valid bucket locations (ADDENDUM) (#6624)

FIPS is only supported in north america AWS regions; relevant tests in
ITestS3AEndpointRegion are skipped for buckets with different endpoints/regions.

Contributed by Viraj Jasani
Viraj Jasani 1 vuosi sitten
vanhempi
commit
9009001143

+ 4 - 3
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 @@ public class DefaultS3ClientFactory extends Configured
     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.

+ 14 - 1
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/connecting.md

@@ -359,8 +359,18 @@ For a single bucket:
 </property>
 ```
 
-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
 ```
@@ -378,6 +388,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.

+ 109 - 16
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java

@@ -48,8 +48,10 @@ 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.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.assume;
 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;
@@ -75,6 +77,12 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
 
   private static final String US_GOV_EAST_1 = "us-gov-east-1";
 
+  private static final String US_REGION_PREFIX = "us-";
+
+  private static final String CA_REGION_PREFIX = "ca-";
+
+  private static final String US_DUAL_STACK_PREFIX = "dualstack.us-";
+
   /**
    * If anyone were ever to create a bucket with this UUID pair it would break the tests.
    */
@@ -156,6 +164,11 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
     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 +181,21 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
 
     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 +226,7 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
     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 +316,8 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
         ENDPOINT,
         AWS_REGION,
         ALLOW_REQUESTER_PAYS,
-        KEY_REQUESTER_PAYS_FILE);
+        KEY_REQUESTER_PAYS_FILE,
+        FIPS_ENDPOINT);
 
     removeBaseAndBucketOverrides(
         DEFAULT_REQUESTER_PAYS_BUCKET_NAME,
@@ -301,20 +325,14 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
         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 +347,8 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
         ENDPOINT,
         AWS_REGION,
         ALLOW_REQUESTER_PAYS,
-        KEY_REQUESTER_PAYS_FILE);
+        KEY_REQUESTER_PAYS_FILE,
+        FIPS_ENDPOINT);
 
     removeBaseAndBucketOverrides(
         DEFAULT_REQUESTER_PAYS_BUCKET_NAME,
@@ -337,15 +356,59 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
         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 +427,39 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
     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 String bucketLocation = getFileSystem().getBucketLocation();
+    assume("FIPS can be enabled to access buckets from US or Canada endpoints only",
+        bucketLocation.startsWith(US_REGION_PREFIX)
+            || bucketLocation.startsWith(CA_REGION_PREFIX)
+            || bucketLocation.startsWith(US_DUAL_STACK_PREFIX));
+
+    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);