[ 
https://issues.apache.org/jira/browse/HADOOP-19044?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17812391#comment-17812391
 ] 

ASF GitHub Bot commented on HADOOP-19044:
-----------------------------------------

steveloughran commented on code in PR #6479:
URL: https://github.com/apache/hadoop/pull/6479#discussion_r1471515567


##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java:
##########
@@ -257,6 +275,65 @@ public void testWithVPCE() throws Throwable {
     expectInterceptorException(client);
   }
 
+  @Test
+  public void testCentralEndpointCrossRegionAccess() throws Throwable {
+    describe("Create bucket on different region and access it using central 
endpoint");
+    final Configuration conf = getConfiguration();
+    removeBaseAndBucketOverrides(conf, ENDPOINT);

Review Comment:
   what should region be set to here? either unset it or explicitly set it.



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java:
##########
@@ -289,17 +290,36 @@ private <BuilderT extends S3BaseClientBuilder<BuilderT, 
ClientT>, ClientT> void
     builder.fipsEnabled(fipsEnabled);
 
     if (endpoint != null) {
+      boolean overrideEndpoint = true;
       checkArgument(!fipsEnabled,
           "%s : %s", ERROR_ENDPOINT_WITH_FIPS, endpoint);
-      builder.endpointOverride(endpoint);
-      // No region was configured, try to determine it from the endpoint.
-      if (region == null) {
-        region = getS3RegionFromEndpoint(parameters.getEndpoint());
+      boolean endpointEndsWithCentral =
+          endpointStr.endsWith(CENTRAL_ENDPOINT);
+      // No region was configured or the endpoint is central,
+      // determine the region from the endpoint.
+      if (region == null || endpointEndsWithCentral) {

Review Comment:
   I don't think anyone should set region=us-west-2 and endpoint = us-west-1 
unless they like debugging things.
   
   all we want is to handle situations where things are not set.
   
   



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java:
##########
@@ -146,7 +150,21 @@ public void testCentralEndpoint() throws Throwable {
     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_EAST_2, false);

Review Comment:
   as in #6466 I'm going to propose we make the static methods accessible and 
unit tests to validate them, because
   * this stuff is so important and complicated we need it running on every pr
   * everyone's ITest setup is different, so may miss things.





> AWS SDK V2 - Update S3A region logic 
> -------------------------------------
>
>                 Key: HADOOP-19044
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19044
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 3.4.0
>            Reporter: Ahmar Suhail
>            Assignee: Viraj Jasani
>            Priority: Major
>              Labels: pull-request-available
>
> If both fs.s3a.endpoint & fs.s3a.endpoint.region are empty, Spark will set 
> fs.s3a.endpoint to 
> s3.amazonaws.com here:
> [https://github.com/apache/spark/blob/9a2f39318e3af8b3817dc5e4baf52e548d82063c/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala#L540]
>  
>  
> HADOOP-18908, updated the region logic such that if fs.s3a.endpoint.region is 
> set, or if a region can be parsed from fs.s3a.endpoint (which will happen in 
> this case, region will be US_EAST_1), cross region access is not enabled. 
> This will cause 400 errors if the bucket is not in US_EAST_1. 
>  
> Proposed: Updated the logic so that if the endpoint is the global 
> s3.amazonaws.com , cross region access is enabled.  
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to