Browse Source

HADOOP-16547. make sure that s3guard prune sets up the FS (#1402). Contributed by Steve Loughran.

Change-Id: Iaf71561cef6c797a3c66fed110faf08da6cac361
Steve Loughran 5 years ago
parent
commit
5db32b8ced

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

@@ -152,8 +152,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
   /**
    * Parse DynamoDB region from either -m option or a S3 path.
    *
-   * This function should only be called from {@link S3GuardTool.Init} or
-   * {@link S3GuardTool.Destroy}.
+   * Note that as a side effect, if the paths included an S3 path,
+   * and there is no region set on the CLI, then the S3A FS is
+   * initialized, after which {@link #filesystem} will no longer
+   * be null.
    *
    * @param paths remaining parameters from CLI.
    * @throws IOException on I/O errors.
@@ -338,6 +340,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
    * @throws ExitUtil.ExitException if the FS is not an S3A FS
    */
   protected void initS3AFileSystem(String path) throws IOException {
+    LOG.debug("Initializing S3A FS to {}", path);
     URI uri = toUri(path);
     // Make sure that S3AFileSystem does not hold an actual MetadataStore
     // implementation.
@@ -363,6 +366,28 @@ public abstract class S3GuardTool extends Configured implements Tool {
     filesystem = (S3AFileSystem) fs;
   }
 
+  /**
+   * Initialize the filesystem if there is none bonded to already and
+   * the command line path list is not empty.
+   * @param paths path list.
+   * @return true if at the end of the call, getFilesystem() is not null
+   * @throws IOException failure to instantiate.
+   */
+  @VisibleForTesting
+  boolean maybeInitFilesystem(final List<String> paths)
+      throws IOException {
+    // is there an S3 FS to create?
+    if (getFilesystem() == null) {
+      // none yet -create one
+      if (!paths.isEmpty()) {
+        initS3AFileSystem(paths.get(0));
+      } else {
+        LOG.debug("No path on command line, so not instantiating FS");
+      }
+    }
+    return getFilesystem() != null;
+  }
+
   /**
    * Parse CLI arguments and returns the position arguments.
    * The options are stored in {@link #commandFormat}.
@@ -592,6 +617,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       // Validate parameters.
       try {
         parseDynamoDBRegion(paths);
+        maybeInitFilesystem(paths);
       } catch (ExitUtil.ExitException e) {
         errorln(USAGE);
         throw e;
@@ -644,6 +670,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
         checkBucketNameOrDDBTableNameProvided(paths);
         checkIfS3BucketIsGuarded(paths);
         parseDynamoDBRegion(paths);
+        maybeInitFilesystem(paths);
       } catch (ExitUtil.ExitException e) {
         errorln(USAGE);
         throw e;
@@ -1064,11 +1091,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
         InterruptedException, IOException {
       List<String> paths = parseArgs(args);
       try {
+        checkBucketNameOrDDBTableNameProvided(paths);
         parseDynamoDBRegion(paths);
       } catch (ExitUtil.ExitException e) {
         errorln(USAGE);
         throw e;
       }
+      maybeInitFilesystem(paths);
       initMetadataStore(false);
 
       Configuration conf = getConf();

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

@@ -27,11 +27,11 @@ import java.io.InputStreamReader;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.UUID;
-import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.fs.s3a.S3AUtils;
@@ -61,6 +61,7 @@ import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_METASTORE_NULL;
 import static org.apache.hadoop.fs.s3a.Constants.S3_METADATA_STORE_IMPL;
 import static org.apache.hadoop.fs.s3a.S3AUtils.clearBucketOption;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_BAD_STATE;
+import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.INVALID_ARGUMENT;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.SUCCESS;
 import static org.apache.hadoop.fs.s3a.s3guard.S3GuardToolTestHelper.exec;
 import static org.apache.hadoop.test.LambdaTestUtils.intercept;
@@ -148,12 +149,7 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
       throws Exception {
     ExitUtil.ExitException ex =
         intercept(ExitUtil.ExitException.class,
-            new Callable<Integer>() {
-              @Override
-              public Integer call() throws Exception {
-                return run(args);
-              }
-            });
+            () -> run(args));
     if (ex.status != status) {
       throw ex;
     }
@@ -333,6 +329,39 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
         "prune", "-" + S3GuardTool.Prune.TOMBSTONE,
         "-seconds", "0",
         testPath.toString());
+    assertNotNull("Command did not create a filesystem",
+        cmd.getFilesystem());
+  }
+
+  /**
+   * HADOOP-16457. In certain cases prune doesn't create an FS.
+   */
+  @Test
+  public void testMaybeInitFilesystem() throws Exception {
+    Path testPath = path("maybeInitFilesystem");
+    S3GuardTool.Prune cmd = new S3GuardTool.Prune(getFileSystem().getConf());
+    cmd.maybeInitFilesystem(Collections.singletonList(testPath.toString()));
+    assertNotNull("Command did not create a filesystem",
+        cmd.getFilesystem());
+  }
+
+  /**
+   * HADOOP-16457. In certain cases prune doesn't create an FS.
+   */
+  @Test
+  public void testMaybeInitFilesystemNoPath() throws Exception {
+    S3GuardTool.Prune cmd = new S3GuardTool.Prune(getFileSystem().getConf());
+    cmd.maybeInitFilesystem(Collections.emptyList());
+    assertNull("Command should not have created a filesystem",
+        cmd.getFilesystem());
+  }
+
+  @Test
+  public void testPruneCommandNoPath() throws Exception {
+    runToFailure(INVALID_ARGUMENT,
+        S3GuardTool.Prune.NAME,
+        "-" + S3GuardTool.Prune.TOMBSTONE,
+        "-seconds", "0");
   }
 
   @Test
@@ -476,7 +505,7 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
     for (Class<? extends S3GuardTool> tool : tools) {
       S3GuardTool cmdR = makeBindedTool(tool);
       describe("Calling " + cmdR.getName() + " without any arguments.");
-      assertExitCode(S3GuardTool.INVALID_ARGUMENT,
+      assertExitCode(INVALID_ARGUMENT,
           intercept(ExitUtil.ExitException.class,
               () -> cmdR.run(new String[]{tool.getName()})));
     }