[
https://issues.apache.org/jira/browse/HADOOP-19044?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17809963#comment-17809963
]
ASF GitHub Bot commented on HADOOP-19044:
-----------------------------------------
ahmarsuhail commented on code in PR #6479:
URL: https://github.com/apache/hadoop/pull/6479#discussion_r1463041828
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java:
##########
@@ -1327,7 +1327,7 @@ private Constants() {
* The special S3 region which can be used to talk to any bucket.
* Value {@value}.
*/
- public static final String AWS_S3_CENTRAL_REGION = "us-east-1";
Review Comment:
let's not change these constants. `AWS_S3_CENTRAL_REGION` should be
us-east-1 as that is the central/global region.
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java:
##########
@@ -354,20 +361,21 @@ 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(String endpoint, 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;
+ // endpoint for central region
+ return Region.of(AWS_S3_CENTRAL_REGION);
Review Comment:
rather than changing constants, return the AWS_S3_DEFAULT_REGION here.
##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3ACrossRegionAccess.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.ContractTestUtils;
+
+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.S3ATestUtils.removeBaseAndBucketOverrides;
+
+/**
+ * Test to verify cross region bucket access.
+ */
+public class ITestS3ACrossRegionAccess extends AbstractS3ATestBase {
+
+ @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);
+
+ try (S3AFileSystem newFs = new S3AFileSystem()) {
Review Comment:
Sorry to nitpick here! but in ITestS3AEndpointRegion, there is also a test
currently that uses the FS, see `testWithoutRegionConfig`.
You can add another test there that does something like:
```
Configuration conf = getConfiguration();
removeBaseAndBucketOverrides(conf, ENDPOINT, AWS_REGION);
conf.set(ENDPOINT, CENTRAL_ENDPOINT);
newFS = new S3AFileSystem();
newFS.initialize(getFileSystem().getUri(), conf);
newFS.create(methodPath()).close();
```
This will fail without your source changes, but passes with them.
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/DefaultS3ClientFactory.java:
##########
@@ -354,20 +361,21 @@ 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(String endpoint, 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;
+ // endpoint for central region
Review Comment:
let's add a detailed comment here now why we have to do this with a link to
the SPARK issue. Something along the lines of "If no region or endpoint is set,
Spark will set the endpoint to s3.amazonaws.com. Since we do not know the
region at this point, use the default region and enable 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]