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

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

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


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java:
##########
@@ -354,20 +360,30 @@ private static URI getS3Endpoint(String endpoint, final 
Configuration conf) {
 
   /**
    * Parses the endpoint to get the region.
-   * If endpoint is the central one, use US_EAST_1.
+   * If endpoint is the central one, use US_EAST_2.
    *
    * @param endpoint the configure endpoint.
+   * @param endpointEndsWithCentral true if the endpoint is configured as 
central.
    * @return the S3 region, null if unable to resolve from endpoint.
    */
-  private static Region getS3RegionFromEndpoint(String endpoint) {
+  private static Region getS3RegionFromEndpoint(final String endpoint,
+      final boolean endpointEndsWithCentral) {
 
-    if(!endpoint.endsWith(CENTRAL_ENDPOINT)) {
+    if (!endpointEndsWithCentral) {
       LOG.debug("Endpoint {} is not the default; parsing", endpoint);
       return AwsHostNameUtils.parseSigningRegion(endpoint, 
S3_SERVICE_NAME).orElse(null);
     }
 
-    // endpoint is for US_EAST_1;
-    return Region.US_EAST_1;
+    // Select default region here to enable cross-region access.

Review Comment:
   mention spark jira and say "spark versions with the changes of 
[SPARK-35878](http://issues.apache.org/jira/browse/SPARK-35878)



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AEndpointRegion.java:
##########
@@ -257,6 +275,33 @@ 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");
+    Configuration conf = getConfiguration();
+    removeBaseAndBucketOverrides(conf, ENDPOINT, AWS_REGION);
+
+    Configuration newConf = new Configuration(conf);
+
+    newConf.set(ENDPOINT, CENTRAL_ENDPOINT);
+
+    newFS = new S3AFileSystem();
+    newFS.initialize(getFileSystem().getUri(), newConf);
+
+    final String file = getMethodName();
+    Path basePath = new Path("basePath-" + getMethodName());

Review Comment:
   use methodPath() as when we finally get the change #5081 in we need to use 
per-test run uniqueness. 



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java:
##########
@@ -294,9 +295,14 @@ private <BuilderT extends S3BaseClientBuilder<BuilderT, 
ClientT>, ClientT> void
       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);
+        region = getS3RegionFromEndpoint(endpointStr, endpointEndsWithCentral);
         if (region != null) {
           origin = "endpoint";
+          if (endpointEndsWithCentral) {
+            builder.crossRegionAccessEnabled(true);

Review Comment:
   add more detail to origin, e,g 'origin with cross-region access"





> 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