4848import static org .apache .hadoop .fs .s3a .Constants .AWS_REGION ;
4949import static org .apache .hadoop .fs .s3a .Constants .CENTRAL_ENDPOINT ;
5050import static org .apache .hadoop .fs .s3a .Constants .ENDPOINT ;
51+ import static org .apache .hadoop .fs .s3a .Constants .FIPS_ENDPOINT ;
5152import static org .apache .hadoop .fs .s3a .Constants .PATH_STYLE_ACCESS ;
5253import static org .apache .hadoop .fs .s3a .DefaultS3ClientFactory .ERROR_ENDPOINT_WITH_FIPS ;
54+ import static org .apache .hadoop .fs .s3a .S3ATestUtils .assume ;
5355import static org .apache .hadoop .fs .s3a .S3ATestUtils .removeBaseAndBucketOverrides ;
5456import static org .apache .hadoop .fs .s3a .test .PublicDatasetTestUtils .DEFAULT_REQUESTER_PAYS_BUCKET_NAME ;
5557import static org .apache .hadoop .io .IOUtils .closeStream ;
@@ -75,6 +77,12 @@ public class ITestS3AEndpointRegion extends AbstractS3ATestBase {
7577
7678 private static final String US_GOV_EAST_1 = "us-gov-east-1" ;
7779
80+ private static final String US_REGION_PREFIX = "us-" ;
81+
82+ private static final String CA_REGION_PREFIX = "ca-" ;
83+
84+ private static final String US_DUAL_STACK_PREFIX = "dualstack.us-" ;
85+
7886 /**
7987 * If anyone were ever to create a bucket with this UUID pair it would break the tests.
8088 */
@@ -156,6 +164,11 @@ public void testCentralEndpoint() throws Throwable {
156164 S3Client client = createS3Client (conf , CENTRAL_ENDPOINT , null , US_EAST_2 , false );
157165
158166 expectInterceptorException (client );
167+
168+ client = createS3Client (conf , CENTRAL_ENDPOINT , null ,
169+ US_EAST_2 , true );
170+
171+ expectInterceptorException (client );
159172 }
160173
161174 @ Test
@@ -168,11 +181,21 @@ public void testCentralEndpointWithRegion() throws Throwable {
168181
169182 expectInterceptorException (client );
170183
184+ client = createS3Client (conf , CENTRAL_ENDPOINT , US_WEST_2 ,
185+ US_WEST_2 , true );
186+
187+ expectInterceptorException (client );
188+
171189 client = createS3Client (conf , CENTRAL_ENDPOINT , US_EAST_1 ,
172190 US_EAST_1 , false );
173191
174192 expectInterceptorException (client );
175193
194+ client = createS3Client (conf , CENTRAL_ENDPOINT , US_EAST_1 ,
195+ US_EAST_1 , true );
196+
197+ expectInterceptorException (client );
198+
176199 }
177200
178201 @ Test
@@ -203,7 +226,7 @@ public void testWithFipsAndEndpoint() throws Throwable {
203226 describe ("Create a client with fips and an endpoint" );
204227
205228 intercept (IllegalArgumentException .class , ERROR_ENDPOINT_WITH_FIPS , () ->
206- createS3Client (getConfiguration (), CENTRAL_ENDPOINT , null , US_EAST_1 , true ));
229+ createS3Client (getConfiguration (), US_WEST_2 , null , US_EAST_1 , true ));
207230 }
208231
209232 @ Test
@@ -293,28 +316,23 @@ public void testCentralEndpointAndDifferentRegionThanBucket() throws Throwable {
293316 ENDPOINT ,
294317 AWS_REGION ,
295318 ALLOW_REQUESTER_PAYS ,
296- KEY_REQUESTER_PAYS_FILE );
319+ KEY_REQUESTER_PAYS_FILE ,
320+ FIPS_ENDPOINT );
297321
298322 removeBaseAndBucketOverrides (
299323 DEFAULT_REQUESTER_PAYS_BUCKET_NAME ,
300324 newConf ,
301325 ENDPOINT ,
302326 AWS_REGION ,
303327 ALLOW_REQUESTER_PAYS ,
304- KEY_REQUESTER_PAYS_FILE );
328+ KEY_REQUESTER_PAYS_FILE ,
329+ FIPS_ENDPOINT );
305330
306331 newConf .set (ENDPOINT , CENTRAL_ENDPOINT );
307332 newConf .set (AWS_REGION , EU_WEST_1 );
308333 newConf .setBoolean (ALLOW_REQUESTER_PAYS , true );
309334
310- Path filePath = new Path (PublicDatasetTestUtils
311- .getRequesterPaysObject (newConf ));
312- newFS = (S3AFileSystem ) filePath .getFileSystem (newConf );
313-
314- Assertions
315- .assertThat (newFS .exists (filePath ))
316- .describedAs ("Existence of path: " + filePath )
317- .isTrue ();
335+ assertRequesterPaysFileExistence (newConf );
318336 }
319337
320338 @ Test
@@ -329,23 +347,68 @@ public void testCentralEndpointAndSameRegionAsBucket() throws Throwable {
329347 ENDPOINT ,
330348 AWS_REGION ,
331349 ALLOW_REQUESTER_PAYS ,
332- KEY_REQUESTER_PAYS_FILE );
350+ KEY_REQUESTER_PAYS_FILE ,
351+ FIPS_ENDPOINT );
333352
334353 removeBaseAndBucketOverrides (
335354 DEFAULT_REQUESTER_PAYS_BUCKET_NAME ,
336355 newConf ,
337356 ENDPOINT ,
338357 AWS_REGION ,
339358 ALLOW_REQUESTER_PAYS ,
340- KEY_REQUESTER_PAYS_FILE );
359+ KEY_REQUESTER_PAYS_FILE ,
360+ FIPS_ENDPOINT );
341361
342362 newConf .set (ENDPOINT , CENTRAL_ENDPOINT );
343363 newConf .set (AWS_REGION , US_WEST_2 );
344364 newConf .setBoolean (ALLOW_REQUESTER_PAYS , true );
345365
366+ assertRequesterPaysFileExistence (newConf );
367+ }
368+
369+ @ Test
370+ public void testCentralEndpointAndFipsForPublicBucket () throws Throwable {
371+ describe ("Access public bucket using central endpoint and region "
372+ + "same as that of the public bucket with fips enabled" );
373+ final Configuration conf = getConfiguration ();
374+ final Configuration newConf = new Configuration (conf );
375+
376+ removeBaseAndBucketOverrides (
377+ newConf ,
378+ ENDPOINT ,
379+ AWS_REGION ,
380+ ALLOW_REQUESTER_PAYS ,
381+ KEY_REQUESTER_PAYS_FILE ,
382+ FIPS_ENDPOINT );
383+
384+ removeBaseAndBucketOverrides (
385+ DEFAULT_REQUESTER_PAYS_BUCKET_NAME ,
386+ newConf ,
387+ ENDPOINT ,
388+ AWS_REGION ,
389+ ALLOW_REQUESTER_PAYS ,
390+ KEY_REQUESTER_PAYS_FILE ,
391+ FIPS_ENDPOINT );
392+
393+ newConf .set (ENDPOINT , CENTRAL_ENDPOINT );
394+ newConf .set (AWS_REGION , US_WEST_2 );
395+ newConf .setBoolean (ALLOW_REQUESTER_PAYS , true );
396+ newConf .setBoolean (FIPS_ENDPOINT , true );
397+
398+ assertRequesterPaysFileExistence (newConf );
399+ }
400+
401+ /**
402+ * Assert that the file exists on the requester pays public bucket.
403+ *
404+ * @param conf the configuration object.
405+ * @throws IOException if file system operations encounter errors.
406+ */
407+ private void assertRequesterPaysFileExistence (Configuration conf )
408+ throws IOException {
346409 Path filePath = new Path (PublicDatasetTestUtils
347- .getRequesterPaysObject (newConf ));
348- newFS = (S3AFileSystem ) filePath .getFileSystem (newConf );
410+ .getRequesterPaysObject (conf ));
411+ newFS = (S3AFileSystem ) filePath .getFileSystem (conf );
349412
350413 Assertions
351414 .assertThat (newFS .exists (filePath ))
@@ -364,9 +427,39 @@ public void testCentralEndpointAndNullRegionWithCRUD() throws Throwable {
364427 removeBaseAndBucketOverrides (
365428 newConf ,
366429 ENDPOINT ,
367- AWS_REGION );
430+ AWS_REGION ,
431+ FIPS_ENDPOINT );
432+
433+ newConf .set (ENDPOINT , CENTRAL_ENDPOINT );
434+
435+ newFS = new S3AFileSystem ();
436+ newFS .initialize (getFileSystem ().getUri (), newConf );
437+
438+ assertOpsUsingNewFs ();
439+ }
440+
441+ @ Test
442+ public void testCentralEndpointAndNullRegionFipsWithCRUD () throws Throwable {
443+ describe ("Access the test bucket using central endpoint and"
444+ + " null region and fips enabled, perform file system CRUD operations" );
445+
446+ final String bucketLocation = getFileSystem ().getBucketLocation ();
447+ assume ("FIPS can be enabled to access buckets from US or Canada endpoints only" ,
448+ bucketLocation .startsWith (US_REGION_PREFIX )
449+ || bucketLocation .startsWith (CA_REGION_PREFIX )
450+ || bucketLocation .startsWith (US_DUAL_STACK_PREFIX ));
451+
452+ final Configuration conf = getConfiguration ();
453+ final Configuration newConf = new Configuration (conf );
454+
455+ removeBaseAndBucketOverrides (
456+ newConf ,
457+ ENDPOINT ,
458+ AWS_REGION ,
459+ FIPS_ENDPOINT );
368460
369461 newConf .set (ENDPOINT , CENTRAL_ENDPOINT );
462+ newConf .setBoolean (FIPS_ENDPOINT , true );
370463
371464 newFS = new S3AFileSystem ();
372465 newFS .initialize (getFileSystem ().getUri (), newConf );
0 commit comments