Selaa lähdekoodia

HADOOP-17261. s3a rename() needs s3:deleteObjectVersion permission (#2303)

Contributed by Steve Loughran.

Change-Id: I8e89a402a24bd9fb958e0fa93d1a28191093851d
Steve Loughran 4 vuotta sitten
vanhempi
commit
4bb9d593da

+ 8 - 10
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RenameOperation.java

@@ -215,12 +215,11 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
    * This method must only be called from the primary thread.
    * @param path path to the object
    * @param key key of the object.
-   * @param version object version.
    */
-  private void queueToDelete(Path path, String key, String version) {
+  private void queueToDelete(Path path, String key) {
     LOG.debug("Queueing to delete {}", path);
     pathsToDelete.add(path);
-    keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key, version));
+    keysToDelete.add(new DeleteObjectsRequest.KeyVersion(key));
   }
 
   /**
@@ -228,28 +227,26 @@ public class RenameOperation extends ExecutingStoreOperation<Long> {
    * <p></p>
    * no-op if the list is empty.
    * <p></p>
-   * See {@link #queueToDelete(Path, String, String)} for
+   * See {@link #queueToDelete(Path, String)} for
    * details on safe use of this method.
    *
    * @param markersToDelete markers
    */
   private void queueToDelete(
       List<DirMarkerTracker.Marker> markersToDelete) {
-    markersToDelete.forEach(m ->
-        queueToDelete(m));
+    markersToDelete.forEach(this::queueToDelete);
   }
 
   /**
    * Queue a single marker for deletion.
    * <p></p>
-   * See {@link #queueToDelete(Path, String, String)} for
+   * See {@link #queueToDelete(Path, String)} for
    * details on safe use of this method.
    *
    * @param marker markers
    */
   private void queueToDelete(final DirMarkerTracker.Marker marker) {
-    queueToDelete(marker.getPath(), marker.getKey(),
-        marker.getStatus().getVersionId());
+    queueToDelete(marker.getPath(), marker.getKey());
   }
 
   /**
@@ -418,6 +415,7 @@ Are   * @throws IOException failure
     while (iterator.hasNext()) {
       // get the next entry in the listing.
       S3ALocatedFileStatus child = iterator.next();
+      LOG.debug("To rename {}", child);
       // convert it to an S3 key.
       String k = storeContext.pathToKey(child.getPath());
       // possibly adding a "/" if it represents directory and it does
@@ -450,7 +448,7 @@ Are   * @throws IOException failure
         Path childDestPath = storeContext.keyToPath(newDestKey);
 
         // mark the source file for deletion on a successful copy.
-        queueToDelete(childSourcePath, key, child.getVersionId());
+        queueToDelete(childSourcePath, key);
           // now begin the single copy
         CompletableFuture<Path> copy = initiateCopy(child, key,
             childSourcePath, newDestKey, childDestPath);

+ 39 - 0
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md

@@ -423,6 +423,45 @@ If the client using [assumed roles](assumed_roles.html), and a policy
 is set in `fs.s3a.assumed.role.policy`, then that policy declares
 _all_ the rights which the caller has.
 
+### <a name="access_denied_renaming"></a> `AccessDeniedException` in rename, "MultiObjectDeleteException: One or more objects could not be deleted"
+
+
+```
+mv: rename s3a://london/dest to s3a://london/src on
+s3a://london/dest:
+    com.amazonaws.services.s3.model.MultiObjectDeleteException: One or more objects
+    could not be deleted (Service: null; Status Code: 200; Error Code: null; Request
+    ID: 5C9018EF245F02C5; S3 Extended Request ID:
+    5fQ2RVCPF0rdvADRv2XY3U4yb2J0gHRID/4jm1eqCXp7RxpU0dH9DliChYsCUD1aVCFtbwfWJWY=),
+    S3 Extended Request ID:
+    5fQ2RVCPF0rdvADRv2XY3U4yb2J0gHRID/4jm1eqCXp7RxpU0dH9DliChYsCUD1aVCFtbwfWJWY=:null:
+    AccessDenied: dest/file10: Access Denied
+```
+
+The S3A connector's emulation of file and directory rename is implemented by copying each file,
+then deleting the originals. This delete process is done in batches, by default in a single
+"multiple object delete request". If one or more of the objects listed in the request cannot
+be deleted, an error is returned in S3 listing which objects were not deleted.
+If the cause was "access denied", it is translated into an `AccessDeniedException`.
+
+The rename is halted at this point: files may be present in both the source and destination directories.
+Those files which could not be deleted from the source directory will also have been copied
+into the destination directory. Files which were successfully deleted from the source
+directory will _only_ be in the destination. And files for which the rename operation
+had yet to commence -they will only be in the source tree.
+
+The user has to recover from this themselves. Be assured: no data will have been deleted, it
+is just that the data may now be scattered across two directories. 
+Note: this is one reason why any application which tries to atomically commit work
+via rename (classic Hadoop output committers, distcp with the `-atomic` option) are
+not safe to use with S3. It is not a file system.
+
+For an 'AccessDenied' failure, the root cause is IAM permissions.
+The user/role/bucket must have the permission
+`s3:DeleteObject` on the source path. It is safest to grant `s3:Delete*` so
+that if a future version of the S3A connector supported extra operations
+(explicit deletion of versioned files, get/set/delete object tagging, ...),
+the client will have the permission to use them.
 
 ### <a name="kms_access_denied"></a>  `AccessDeniedException` when using SSE-KMS
 

+ 45 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
 import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
 import org.apache.hadoop.util.DurationInfo;
 
@@ -839,4 +840,48 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
   public static String filenameOfIndex(final int i) {
     return String.format("%s%03d", PREFIX, i);
   }
+
+  /**
+   * Verifies that s3:DeleteObjectVersion is not required for rename.
+   * <p></p>
+   * See HADOOP-17621.
+   * <p></p>
+   * This test will only show a regression if the bucket has versioning
+   * enabled *and* S3Guard is enabled.
+   */
+  @Test
+  public void testRenamePermissionRequirements() throws Throwable {
+    describe("Verify rename() only needs s3:DeleteObject permission");
+    // close the existing roleFS
+    IOUtils.cleanupWithLogger(LOG, roleFS);
+
+    // create an assumed role config which doesn't have
+    // s3:DeleteObjectVersion permission, and attempt rename
+    // and then delete.
+    Configuration roleConfig = createAssumedRoleConfig();
+    bindRolePolicyStatements(roleConfig,
+        STATEMENT_S3GUARD_CLIENT,
+        STATEMENT_ALLOW_SSE_KMS_RW,
+        STATEMENT_ALL_BUCKET_READ_ACCESS,  // root:     r-x
+        new Statement(Effects.Allow)       // dest:     rwx
+            .addActions(S3_PATH_RW_OPERATIONS)
+            .addResources(directory(basePath)),
+        new Statement(Effects.Deny)
+            .addActions(S3_DELETE_OBJECT_VERSION)
+            .addResources(directory(basePath)));
+    roleFS = (S3AFileSystem) basePath.getFileSystem(roleConfig);
+
+    Path srcDir = new Path(basePath, "src");
+    Path destDir = new Path(basePath, "dest");
+    roleFS.mkdirs(srcDir);
+
+    // the role FS has everything but that deleteObjectVersion permission, so
+    // MUST be able to create files
+    List<Path> createdFiles = createFiles(roleFS, srcDir, dirDepth, fileCount,
+        dirCount);
+    roleFS.rename(srcDir, destDir);
+    roleFS.rename(destDir, srcDir);
+    roleFS.delete(srcDir, true);
+
+  }
 }