Browse Source

HADOOP-15430. hadoop fs -mkdir -p path-ending-with-slash/ fails with s3guard (#1646)

Contributed by Steve Loughran

* move qualify logic to S3AFileSystem.makeQualified()
* make S3AFileSystem.qualify() a private redirect to that
* ITestS3GuardFsShell turned off
Steve Loughran 5 years ago
parent
commit
0a9b3c98b1

+ 25 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -992,6 +992,8 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
 
   /**
    * Convert a key to a fully qualified path.
+   * This includes fixing up the URI so that if it ends with a trailing slash,
+   * that is corrected, similar to {@code Path.normalizePath()}.
    * @param key input key
    * @return the fully qualified path including URI scheme and bucket name.
    */
@@ -999,13 +1001,35 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities,
     return qualify(keyToPath(key));
   }
 
+  @Override
+  public Path makeQualified(final Path path) {
+    Path q = super.makeQualified(path);
+    if (!q.isRoot()) {
+      String urlString = q.toUri().toString();
+      if (urlString.endsWith(Path.SEPARATOR)) {
+        // this is a path which needs root stripping off to avoid
+        // confusion, See HADOOP-15430
+        LOG.debug("Stripping trailing '/' from {}", q);
+        // deal with an empty "/" at the end by mapping to the parent and
+        // creating a new path from it
+        q = new Path(urlString.substring(0, urlString.length() - 1));
+      }
+    }
+    if (!q.isRoot() && q.getName().isEmpty()) {
+      q = q.getParent();
+    }
+    return q;
+  }
+
   /**
    * Qualify a path.
+   * This includes fixing up the URI so that if it ends with a trailing slash,
+   * that is corrected, similar to {@code Path.normalizePath()}.
    * @param path path to qualify
    * @return a qualified path.
    */
   public Path qualify(Path path) {
-    return path.makeQualified(uri, workingDir);
+    return makeQualified(path);
   }
 
   /**

+ 14 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/PathMetadataDynamoDBTranslation.java

@@ -318,8 +318,20 @@ public final class PathMetadataDynamoDBTranslation {
   static PrimaryKey pathToKey(Path path) {
     Preconditions.checkArgument(!path.isRoot(),
         "Root path is not mapped to any PrimaryKey");
-    return new PrimaryKey(PARENT, pathToParentKey(path.getParent()), CHILD,
-        path.getName());
+    String childName = path.getName();
+    PrimaryKey key = new PrimaryKey(PARENT,
+        pathToParentKey(path.getParent()), CHILD,
+        childName);
+    for (KeyAttribute attr : key.getComponents()) {
+      String name = attr.getName();
+      Object v = attr.getValue();
+      Preconditions.checkNotNull(v,
+          "Null value for DynamoDB attribute \"%s\"", name);
+      Preconditions.checkState(!((String)v).isEmpty(),
+          "Empty string value for DynamoDB attribute \"%s\"", name);
+    }
+    return key;
+
   }
 
   /**

+ 2 - 2
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/s3guard/S3GuardTool.java

@@ -1019,7 +1019,7 @@ public abstract class S3GuardTool extends Configured implements Tool,
      * @throws IOException on I/O errors.
      */
     private void compareRoot(Path path, PrintStream out) throws IOException {
-      Path qualified = getFilesystem().qualify(path);
+      Path qualified = getFilesystem().makeQualified(path);
       FileStatus s3Status = null;
       try {
         s3Status = getFilesystem().getFileStatus(qualified);
@@ -1050,7 +1050,7 @@ public abstract class S3GuardTool extends Configured implements Tool,
       } else {
         root = new Path(uri.getPath());
       }
-      root = getFilesystem().qualify(root);
+      root = getFilesystem().makeQualified(root);
       compareRoot(root, out);
       out.flush();
       return SUCCESS;

+ 114 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.fs.s3a;
 import java.io.ByteArrayInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.net.URI;
 import java.nio.charset.StandardCharsets;
 
 import com.amazonaws.services.s3.model.ObjectMetadata;
@@ -254,4 +255,117 @@ public class ITestS3AMiscOperations extends AbstractS3ATestBase {
     }
   }
 
+  /**
+   * Verify that paths with a trailing "/" are fixed up.
+   */
+  @Test
+  public void testPathFixup() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    Path path = fs.makeQualified(new Path("path"));
+    String trailing = path.toUri().toString() + "/";
+    verifyNoTrailingSlash("path from string",
+        new Path(trailing));
+
+    // here the problem: the URI constructor doesn't strip trailing "/" chars
+    URI trailingURI = verifyTrailingSlash("trailingURI", new URI(trailing));
+    Path pathFromTrailingURI =
+        verifyTrailingSlash("pathFromTrailingURI", new Path(trailingURI));
+
+    // here is the fixup
+    verifyNoTrailingSlash(
+        "path from fs.makeQualified()",
+        fs.makeQualified(pathFromTrailingURI));
+  }
+
+  /**
+   * Verify that paths with a trailing "//" are fixed up.
+   */
+  @Test
+  public void testPathDoubleSlashFixup() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    Path path = fs.makeQualified(new Path("path"));
+    String trailing2 = path.toUri().toString() + "//";
+    verifyNoTrailingSlash("path from string",
+        new Path(trailing2));
+
+    // here the problem: the URI constructor doesn't strip trailing "/" chars
+    URI trailingURI = new URI(trailing2);
+    Path pathFromTrailingURI =
+        verifyTrailingSlash("pathFromTrailingURI", new Path(trailingURI));
+
+    // here is the fixup
+    verifyNoTrailingSlash(
+        "path from fs.makeQualified()",
+        fs.makeQualified(pathFromTrailingURI));
+  }
+
+  /**
+   * Verify that root path fixup does retain any trailing "/", because
+   * that matters.
+   */
+  @Test
+  public void testRootPathFixup() throws Throwable {
+    final S3AFileSystem fs = getFileSystem();
+    // fs.getURI() actually returns a path without any trailing /
+    String baseFsURI = fs.getUri().toString();
+    Path rootPath_from_FS_URI = verifyNoTrailingSlash("root", new Path(baseFsURI));
+
+    // add a single / to a string
+    String trailing = verifyTrailingSlash("FS URI",
+        baseFsURI + "/");
+    Path root_path_from_trailing_string =
+        verifyTrailingSlash("root path from string", new Path(trailing));
+
+    // now verify that the URI constructor retrains that /
+    URI trailingURI = verifyTrailingSlash("trailingURI", new URI(trailing));
+    Path pathFromTrailingURI =
+        verifyTrailingSlash("pathFromTrailingURI", new Path(trailingURI));
+
+    // Root path fixup is expected to retain that trailing /
+    Path pathFromQualify = verifyTrailingSlash(
+        "path from fs.makeQualified()",
+        fs.makeQualified(pathFromTrailingURI));
+    assertEquals(root_path_from_trailing_string, pathFromQualify);
+
+    // and if you fix up the root path without a string, you get
+    // back a root path without a string
+    Path pathFromRootQualify = verifyNoTrailingSlash(
+        "path from fs.makeQualified(" + baseFsURI +")",
+        fs.makeQualified(rootPath_from_FS_URI));
+
+    assertEquals(rootPath_from_FS_URI, pathFromRootQualify);
+    assertNotEquals(rootPath_from_FS_URI, root_path_from_trailing_string);
+  }
+
+  /**
+   * Verify that an object's string value path has a single trailing / symbol;
+   * returns the object.
+   * @param role role for error messages
+   * @param o object
+   * @param <T> type of object
+   * @return the object.
+   */
+  private static <T> T verifyTrailingSlash(String role, T o) {
+    String s = o.toString();
+    assertTrue(role + " lacks trailing slash " + s,
+        s.endsWith("/"));
+    assertFalse(role + " has double trailing slash " + s,
+        s.endsWith("//"));
+    return o;
+  }
+
+  /**
+   * Verify that an object's string value path has no trailing / symbol;
+   * returns the object.
+   * @param role role for error messages
+   * @param o object
+   * @param <T> type of object
+   * @return the object.
+   */
+  private static <T> T verifyNoTrailingSlash(String role, T o) {
+    String s = o.toString();
+    assertFalse(role + " has trailing slash " + s,
+        s.endsWith("/"));
+    return o;
+  }
 }

+ 1 - 1
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/AbstractS3GuardToolTestBase.java

@@ -248,7 +248,7 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
       ContractTestUtils.touch(fs, path);
     } else if (onMetadataStore) {
       S3AFileStatus status = new S3AFileStatus(100L, System.currentTimeMillis(),
-          fs.qualify(path), 512L, "hdfs", null, null);
+          fs.makeQualified(path), 512L, "hdfs", null, null);
       putFile(ms, status);
     }
   }

+ 144 - 0
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/ITestS3GuardFsShell.java

@@ -0,0 +1,144 @@
+/*
+ * 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.s3guard;
+
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FsShell;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
+import org.apache.hadoop.util.ToolRunner;
+
+/**
+ * Test FS shell and S3Guard (and of course, the rest of S3).
+ */
+public class ITestS3GuardFsShell extends AbstractS3ATestBase {
+
+  /**
+   * Run a shell command.
+   * @param args array of arguments.
+   * @return the exit code.
+   * @throws Exception any exception raised.
+   */
+  private int fsShell(String[] args) throws Exception {
+    FsShell shell = new FsShell(getConfiguration());
+    try {
+      return ToolRunner.run(shell, args);
+    } finally {
+      shell.close();
+    }
+  }
+
+  /**
+   * Execute a command and verify that it returned the specific exit code.
+   * @param expected expected value
+   * @param args array of arguments.
+   * @throws Exception any exception raised.
+   */
+  private void exec(int expected, String[] args) throws Exception {
+    int result = fsShell(args);
+    String argslist = Arrays.stream(args).collect(Collectors.joining(" "));
+    assertEquals("hadoop fs " + argslist, expected, result);
+  }
+
+  /**
+   * Execute a shell command expecting a result of 0.
+   * @param args array of arguments.
+   * @throws Exception any exception raised.
+   */
+  private void exec(String[] args) throws Exception {
+    exec(0, args);
+  }
+
+  /**
+   * Issue a mkdir without a trailing /.
+   */
+  @Test
+  public void testMkdirNoTrailing() throws Throwable {
+    Path dest = path("normal");
+    try {
+      String destStr = dest.toString();
+      mkdirs(destStr);
+      isDir(destStr);
+      rmdir(destStr);
+      isNotFound(destStr);
+    } finally {
+      getFileSystem().delete(dest, true);
+    }
+  }
+
+  /**
+   * Issue a mkdir with a trailing /.
+   */
+  @Test
+  public void testMkdirTrailing() throws Throwable {
+    Path base = path("trailing");
+    getFileSystem().delete(base, true);
+    try {
+      String destStr = base.toString() + "/";
+      mkdirs(destStr);
+      isDir(destStr);
+      isDir(base.toString());
+      rmdir(destStr);
+      isNotFound(destStr);
+    } finally {
+      getFileSystem().delete(base, true);
+    }
+  }
+
+  /**
+   * Create the destination path and then call mkdir, expect it to still work.
+   */
+  @Test
+  public void testMkdirTrailingExists() throws Throwable {
+    Path base = path("trailingexists");
+    getFileSystem().mkdirs(base);
+    try {
+      String destStr = base.toString() + "/";
+      // the path already exists
+      isDir(destStr);
+      mkdirs(destStr);
+      isDir(destStr);
+      rmdir(base.toString());
+      isNotFound(destStr);
+    } finally {
+      getFileSystem().delete(base, true);
+    }
+  }
+
+  private void isNotFound(final String destStr) throws Exception {
+    exec(1, new String[]{"-test", "-d", destStr});
+  }
+
+  private void mkdirs(final String destStr) throws Exception {
+    exec(new String[]{"-mkdir", "-p", destStr});
+  }
+
+  private void rmdir(final String destStr) throws Exception {
+    exec(new String[]{"-rmdir", destStr});
+  }
+
+  private void isDir(final String destStr) throws Exception {
+    exec(new String[]{"-test", "-d", destStr});
+  }
+
+}

+ 9 - 6
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/s3guard/TestPathMetadataDynamoDBTranslation.java

@@ -243,16 +243,19 @@ public class TestPathMetadataDynamoDBTranslation extends Assert {
   @Test
   public void testPathToKey() throws Exception {
     LambdaTestUtils.intercept(IllegalArgumentException.class,
-        new Callable<PrimaryKey>() {
-          @Override
-          public PrimaryKey call() throws Exception {
-            return pathToKey(new Path("/"));
-          }
-        });
+        () -> pathToKey(new Path("/")));
     doTestPathToKey(TEST_DIR_PATH);
     doTestPathToKey(TEST_FILE_PATH);
   }
 
+  @Test
+  public void testPathToKeyTrailing() throws Exception {
+    doTestPathToKey(new Path("s3a://test-bucket/myDir/trailing//"));
+    doTestPathToKey(new Path("s3a://test-bucket/myDir/trailing/"));
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> pathToKey(new Path("s3a://test-bucket//")));
+  }
+
   private static void doTestPathToKey(Path path) {
     final PrimaryKey key = pathToKey(path);
     assertNotNull(key);