You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by st...@apache.org on 2019/10/16 08:41:49 UTC

[hadoop] branch trunk updated: HADOOP-16478. S3Guard bucket-info fails if the caller lacks s3:GetBucketLocation.

This is an automated email from the ASF dual-hosted git repository.

stevel pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new bbcf0b9  HADOOP-16478. S3Guard bucket-info fails if the caller lacks s3:GetBucketLocation.
bbcf0b9 is described below

commit bbcf0b91d6f5eb697d09e45505b0e72e193c3d75
Author: Steve Loughran <st...@apache.org>
AuthorDate: Wed Oct 16 09:41:33 2019 +0100

    HADOOP-16478. S3Guard bucket-info fails if the caller lacks s3:GetBucketLocation.
    
    Contributed by Steve Loughran.
    
    Includes HADOOP-16651. S3 getBucketLocation() can return "US" for us-east.
    
    Change-Id: Ifc0dca76e51495ed1a8fc0f077b86bf125deff40
---
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java    | 13 +++--
 .../hadoop/fs/s3a/impl/ContextAccessors.java       |  2 +
 .../apache/hadoop/fs/s3a/impl/NetworkBinding.java  | 15 ++++++
 .../fs/s3a/s3guard/DynamoDBMetadataStore.java      | 11 ++--
 .../apache/hadoop/fs/s3a/s3guard/S3GuardTool.java  | 26 +++++++--
 .../apache/hadoop/fs/s3a/auth/ITestAssumeRole.java | 34 ++++++++++++
 .../hadoop/fs/s3a/auth/ITestCustomSigner.java      |  7 +--
 .../hadoop/fs/s3a/impl/TestNeworkBinding.java      | 61 ++++++++++++++++++++++
 8 files changed, 151 insertions(+), 18 deletions(-)

diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index ff2ba14..6fc3dbe 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -168,6 +168,7 @@ import static org.apache.hadoop.fs.s3a.auth.RolePolicies.allowS3Operations;
 import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.TokenIssuingPolicy.NoTokensAvailable;
 import static org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens.hasDelegationTokenBinding;
 import static org.apache.hadoop.fs.s3a.impl.InternalConstants.SC_404;
+import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 
 /**
@@ -715,6 +716,7 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   /**
    * Get the region of a bucket.
    * @return the region in which a bucket is located
+   * @throws AccessDeniedException if the caller lacks permission.
    * @throws IOException on any failure.
    */
   @Retries.RetryTranslated
@@ -723,17 +725,22 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
   }
 
   /**
-   * Get the region of a bucket.
+   * Get the region of a bucket; fixing up the region so it can be used
+   * in the builders of other AWS clients.
+   * Requires the caller to have the AWS role permission
+   * {@code s3:GetBucketLocation}.
    * Retry policy: retrying, translated.
    * @param bucketName the name of the bucket
    * @return the region in which a bucket is located
+   * @throws AccessDeniedException if the caller lacks permission.
    * @throws IOException on any failure.
    */
   @VisibleForTesting
   @Retries.RetryTranslated
   public String getBucketLocation(String bucketName) throws IOException {
-    return invoker.retry("getBucketLocation()", bucketName, true,
-        ()-> s3.getBucketLocation(bucketName));
+    final String region = invoker.retry("getBucketLocation()", bucketName, true,
+        () -> s3.getBucketLocation(bucketName));
+    return fixBucketRegion(region);
   }
 
   /**
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java
index 1ca3a42..b10cc6d 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/ContextAccessors.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.s3a.impl;
 
 import java.io.File;
 import java.io.IOException;
+import java.nio.file.AccessDeniedException;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.Retries;
@@ -67,6 +68,7 @@ public interface ContextAccessors {
    * Get the region of a bucket. This may be via an S3 API call if not
    * already cached.
    * @return the region in which a bucket is located
+   * @throws AccessDeniedException if the caller lacks permission.
    * @throws IOException on any failure.
    */
   @Retries.RetryTranslated
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java
index c255127..4cc2bfa 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/NetworkBinding.java
@@ -110,4 +110,19 @@ public class NetworkBinding {
               AWS_SOCKET_FACTORY_CLASSNAME, SSL_CHANNEL_MODE, e);
     }
   }
+
+  /**
+   * Given an S3 bucket region as returned by a bucket location query,
+   * fix it into a form which can be used by other AWS commands.
+   * https://forums.aws.amazon.com/thread.jspa?messageID=796829&tstart=0
+   * See also {@code com.amazonaws.services.s3.model.Region.fromValue()}
+   * for its conversion logic.
+   * @param region region from S3 call.
+   * @return the region to use in DDB etc.
+   */
+  public static String fixBucketRegion(final String region) {
+    return region == null || region.equals("US")
+        ? "us-east-1"
+        : region;
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
index 044f3a5..76eb5ca 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/DynamoDBMetadataStore.java
@@ -399,12 +399,11 @@ public class DynamoDBMetadataStore implements MetadataStore,
       } catch (AccessDeniedException e) {
         // access denied here == can't call getBucket. Report meaningfully
         URI uri = owner.getUri();
-        LOG.error("Failed to get bucket location from S3 bucket {}",
-            uri);
-        throw (IOException)new AccessDeniedException(
-            "S3 client role lacks permission "
-                + RolePolicies.S3_GET_BUCKET_LOCATION + " for " + uri)
-            .initCause(e);
+        String message =
+            "Failed to get bucket location as client lacks permission "
+                + RolePolicies.S3_GET_BUCKET_LOCATION + " for " + uri;
+        LOG.error(message);
+        throw (IOException)new AccessDeniedException(message).initCause(e);
       }
       LOG.debug("Inferring DynamoDB region from S3 bucket: {}", region);
     }
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
index bd834e0..109b33c 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.nio.file.AccessDeniedException;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -52,6 +53,7 @@ import org.apache.hadoop.fs.s3a.S3AFileStatus;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
 import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;
 import org.apache.hadoop.fs.s3a.S3AUtils;
+import org.apache.hadoop.fs.s3a.auth.RolePolicies;
 import org.apache.hadoop.fs.s3a.auth.delegation.S3ADelegationTokens;
 import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.s3a.select.SelectTool;
@@ -431,7 +433,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * Run the tool, capturing the output (if the tool supports that).
    *
    * As well as returning an exit code, the implementations can choose to
-   * throw an instance of {@link ExitUtil.ExitException} with their exit
+   * throw an instance of {@code ExitUtil.ExitException} with their exit
    * code set to the desired exit value. The exit code of such an exception
    * is used for the tool's exit code, and the stack trace only logged at
    * debug.
@@ -1147,7 +1149,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
   /**
    * Get info about a bucket and its S3Guard integration status.
    */
-  static class BucketInfo extends S3GuardTool {
+  public static class BucketInfo extends S3GuardTool {
     public static final String NAME = "bucket-info";
     public static final String GUARDED_FLAG = "guarded";
     public static final String UNGUARDED_FLAG = "unguarded";
@@ -1169,7 +1171,15 @@ public abstract class S3GuardTool extends Configured implements Tool {
         + "  -" + ENCRYPTION_FLAG
         + " -require {none, sse-s3, sse-kms} - Require encryption policy";
 
-    BucketInfo(Configuration conf) {
+    /**
+     * Output when the client cannot get the location of a bucket.
+     */
+    @VisibleForTesting
+    public static final String LOCATION_UNKNOWN =
+        "Location unknown -caller lacks "
+            + RolePolicies.S3_GET_BUCKET_LOCATION + " permission";
+
+    public BucketInfo(Configuration conf) {
       super(conf, GUARDED_FLAG, UNGUARDED_FLAG, AUTH_FLAG, NONAUTH_FLAG, MAGIC_FLAG);
       CommandFormat format = getCommandFormat();
       format.addOptionWithValue(ENCRYPTION_FLAG);
@@ -1212,7 +1222,15 @@ public abstract class S3GuardTool extends Configured implements Tool {
       URI fsUri = fs.getUri();
       MetadataStore store = fs.getMetadataStore();
       println(out, "Filesystem %s", fsUri);
-      println(out, "Location: %s", fs.getBucketLocation());
+      try {
+        println(out, "Location: %s", fs.getBucketLocation());
+      } catch (AccessDeniedException e) {
+        // Caller cannot get the location of this bucket due to permissions
+        // in their role or the bucket itself.
+        // Note and continue.
+        LOG.debug("failed to get bucket location", e);
+        println(out, LOCATION_UNKNOWN);
+      }
       boolean usingS3Guard = !(store instanceof NullMetadataStore);
       boolean authMode = false;
       if (usingS3Guard) {
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
index 7261353..a4c6a6e 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java
@@ -29,6 +29,7 @@ import java.util.stream.IntStream;
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.services.securitytoken.model.AWSSecurityTokenServiceException;
 import com.fasterxml.jackson.core.JsonProcessingException;
+import org.assertj.core.api.Assertions;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -50,6 +51,7 @@ import org.apache.hadoop.fs.s3a.commit.CommitConstants;
 import org.apache.hadoop.fs.s3a.commit.CommitOperations;
 import org.apache.hadoop.fs.s3a.commit.files.PendingSet;
 import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
+import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool;
 
 import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
 import static org.apache.hadoop.fs.s3a.Constants.*;
@@ -60,6 +62,7 @@ import static org.apache.hadoop.fs.s3a.auth.RoleModel.*;
 import static org.apache.hadoop.fs.s3a.auth.RolePolicies.*;
 import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.forbidden;
 import static org.apache.hadoop.fs.s3a.auth.RoleTestUtils.newAssumedRoleConfig;
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec;
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
 import static org.apache.hadoop.test.LambdaTestUtils.*;
 
@@ -722,4 +725,35 @@ public class ITestAssumeRole extends AbstractS3ATestBase {
         roleFS.delete(pathWhichDoesntExist, true));
   }
 
+  /**
+   * Block access to bucket locations and verify that {@code getBucketLocation}
+   * fails -but that the bucket-info command recovers from this.
+   */
+  @Test
+  public void testBucketLocationForbidden() throws Throwable {
+
+    describe("Restrict role to read only");
+    Configuration conf = createAssumedRoleConfig();
+
+    // S3Guard is turned off so that it isn't trying to work out
+    // where any table is.
+    removeBaseAndBucketOverrides(getTestBucketName(conf), conf,
+        S3_METADATA_STORE_IMPL);
+
+    bindRolePolicyStatements(conf,
+        STATEMENT_S3GUARD_CLIENT,
+        STATEMENT_ALLOW_SSE_KMS_RW,
+        statement(true, S3_ALL_BUCKETS, S3_ALL_OPERATIONS),
+        statement(false, S3_ALL_BUCKETS, S3_GET_BUCKET_LOCATION));
+    Path path = methodPath();
+    roleFS = (S3AFileSystem) path.getFileSystem(conf);
+    forbidden("",
+        () -> roleFS.getBucketLocation());
+    S3GuardTool.BucketInfo infocmd = new S3GuardTool.BucketInfo(conf);
+    URI fsUri = getFileSystem().getUri();
+    String info = exec(infocmd, S3GuardTool.BucketInfo.NAME,
+        fsUri.toString());
+    Assertions.assertThat(info)
+        .contains(S3GuardTool.BucketInfo.LOCATION_UNKNOWN);
+  }
 }
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java
index 651cdad..2e13deb 100644
--- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestCustomSigner.java
@@ -47,6 +47,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 
 import static org.apache.hadoop.fs.s3a.Constants.CUSTOM_SIGNERS;
 import static org.apache.hadoop.fs.s3a.Constants.SIGNING_ALGORITHM_S3;
+import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
 
 /**
  * Tests for custom Signers and SignerInitializers.
@@ -137,11 +138,7 @@ public class ITestCustomSigner extends AbstractS3ATestBase {
         .withForceGlobalBucketAccessEnabled(true).withRegion("us-east-1")
         .build();
     String region = s3.getBucketLocation(bucketName);
-    //  See: https://forums.aws.amazon.com/thread.jspa?messageID=796829&tstart=0
-    if (region.equals("US")) {
-      region = "us-east-1";
-    }
-    return region;
+    return fixBucketRegion(region);
   }
 
   @Private
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java
new file mode 100644
index 0000000..eebc3bf
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestNeworkBinding.java
@@ -0,0 +1,61 @@
+/*
+ * 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.impl;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.test.HadoopTestBase;
+
+import static org.apache.hadoop.fs.s3a.impl.NetworkBinding.fixBucketRegion;
+
+/**
+ * Unit tests related to the {@link NetworkBinding} class.
+ */
+public class TestNeworkBinding extends HadoopTestBase {
+
+  private static final String US_EAST_1 = "us-east-1";
+  private static final String US_WEST_2 = "us-west-2";
+
+  @Test
+  public void testUSEast() {
+    assertRegionFixup(US_EAST_1, US_EAST_1);
+  }
+
+  @Test
+  public void testUSWest() {
+    assertRegionFixup(US_WEST_2, US_WEST_2);
+  }
+
+  @Test
+  public void testRegionUStoUSEast() {
+    assertRegionFixup("US", US_EAST_1);
+  }
+
+  @Test
+  public void testRegionNullToUSEast() {
+    assertRegionFixup(null, US_EAST_1);
+  }
+
+  private static void assertRegionFixup(String region, String expected) {
+    Assertions.assertThat(fixBucketRegion(region))
+        .describedAs("Fixup of %s", region)
+        .isEqualTo(expected);
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org