Browse Source

Revert "HADOOP-15843. s3guard bucket-info command to not print a stack trace on bucket-not-found."

This reverts commit c4a00d1ad3d3cfc02a6a4e1e04353678f2d588e1.
Steve Loughran 6 years ago
parent
commit
920a89627d

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

@@ -69,7 +69,6 @@ import static org.apache.hadoop.service.launcher.LauncherExitCodes.*;
  * CLI to manage S3Guard Metadata Store.
  */
 public abstract class S3GuardTool extends Configured implements Tool {
-
   private static final Logger LOG = LoggerFactory.getLogger(S3GuardTool.class);
 
   private static final String NAME = "s3guard";
@@ -96,9 +95,6 @@ public abstract class S3GuardTool extends Configured implements Tool {
   private static final String DATA_IN_S3_IS_PRESERVED
       = "(all data in S3 is preserved)";
 
-  public static final String E_NO_METASTORE_OR_FILESYSTEM
-      = "No metastore or filesystem specified";
-
   public abstract String getUsage();
 
   // Exit codes
@@ -279,8 +275,12 @@ public abstract class S3GuardTool extends Configured implements Tool {
     if (getStore() != null) {
       return getStore();
     }
-    final boolean hasFileSystem = filesystem != null;
-    final Configuration conf = hasFileSystem ? filesystem.getConf() : getConf();
+    Configuration conf;
+    if (filesystem == null) {
+      conf = getConf();
+    } else {
+      conf = filesystem.getConf();
+    }
     String metaURI = getCommandFormat().getOptValue(META_FLAG);
     if (metaURI != null && !metaURI.isEmpty()) {
       URI uri = URI.create(metaURI);
@@ -302,13 +302,6 @@ public abstract class S3GuardTool extends Configured implements Tool {
             String.format("Metadata store %s is not supported", uri));
       }
     } else {
-      if (!hasFileSystem) {
-        // command didn't declare a metadata store URI or a bucket.
-        // to avoid problems related to picking up a shared table for actions
-        // line init and destroy (HADOOP-15843), this is rejected
-        printHelp(this);
-        throw usageError(E_NO_METASTORE_OR_FILESYSTEM);
-      }
       // CLI does not specify metadata store URI, it uses default metadata store
       // DynamoDB instead.
       setStore(new DynamoDBMetadataStore());
@@ -317,10 +310,10 @@ public abstract class S3GuardTool extends Configured implements Tool {
       }
     }
 
-    if (hasFileSystem) {
-      getStore().initialize(filesystem);
-    } else {
+    if (filesystem == null) {
       getStore().initialize(conf);
+    } else {
+      getStore().initialize(filesystem);
     }
     LOG.info("Metadata store {} is initialized.", getStore());
     return getStore();
@@ -1463,13 +1456,13 @@ public abstract class S3GuardTool extends Configured implements Tool {
     return uri;
   }
 
-  protected static void printHelp(S3GuardTool tool) {
-    if (tool == null) {
+  private static void printHelp() {
+    if (command == null) {
       errorln("Usage: hadoop " + USAGE);
       errorln("\tperform S3Guard metadata store " +
           "administrative commands.");
     } else {
-      errorln("Usage: hadoop " + tool.getUsage());
+      errorln("Usage: hadoop " + command.getUsage());
     }
     errorln();
     errorln(COMMON_USAGE);
@@ -1511,6 +1504,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
     }
   }
 
+
   /**
    * Handle store not found by converting to an exit exception
    * with specific error code.
@@ -1572,18 +1566,6 @@ public abstract class S3GuardTool extends Configured implements Tool {
         String.format(format, args));
   }
 
-
-  /**
-   * Build the exception to raise on a usage error
-   * @param format string format
-   * @param args optional arguments for the string
-   * @return a new exception to throw
-   */
-  protected static ExitUtil.ExitException usageError(
-      String format, Object...args) {
-    return new ExitUtil.ExitException(E_USAGE, String.format(format, args));
-  }
-
   /**
    * Execute the command with the given arguments.
    *
@@ -1599,8 +1581,8 @@ public abstract class S3GuardTool extends Configured implements Tool {
     String[] otherArgs = new GenericOptionsParser(conf, args)
         .getRemainingArgs();
     if (otherArgs.length == 0) {
-      printHelp(null);
-      throw usageError("No arguments provided");
+      printHelp();
+      throw new ExitUtil.ExitException(E_USAGE, "No arguments provided");
     }
     final String subCommand = otherArgs[0];
     LOG.debug("Executing command {}", subCommand);
@@ -1635,7 +1617,7 @@ public abstract class S3GuardTool extends Configured implements Tool {
       command = new SelectTool(conf);
       break;
     default:
-      printHelp(null);
+      printHelp();
       throw new ExitUtil.ExitException(E_USAGE,
           "Unknown command " + subCommand);
     }
@@ -1652,17 +1634,11 @@ public abstract class S3GuardTool extends Configured implements Tool {
       exit(ret, "");
     } catch (CommandFormat.UnknownOptionException e) {
       errorln(e.getMessage());
-      printHelp(command);
+      printHelp();
       exit(E_USAGE, e.getMessage());
     } catch (ExitUtil.ExitException e) {
       // explicitly raised exit code
       exit(e.getExitCode(), e.toString());
-    } catch (FileNotFoundException e) {
-      // bucket doesn't exist or similar.
-      // skip the stack trace and choose the return code of 44, "404"
-      errorln(e.toString());
-      LOG.debug("Not found:", e);
-      exit(EXIT_NOT_FOUND, e.toString());
     } catch (Throwable e) {
       e.printStackTrace(System.err);
       exit(ERROR, e.toString());

+ 0 - 14
hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/s3guard.md

@@ -1006,20 +1006,6 @@ There's are limit on how often you can change the capacity of an DynamoDB table;
 if you call set-capacity too often, it fails. Wait until the after the time indicated
 and try again.
 
-### Error `Invalid region specified`
-
-```
-java.io.IOException: Invalid region specified "iceland-2":
-  Region can be configured with fs.s3a.s3guard.ddb.region:
-  us-gov-west-1, us-east-1, us-east-2, us-west-1, us-west-2,
-  eu-west-1, eu-west-2, eu-west-3, eu-central-1, ap-south-1,
-  ap-southeast-1, ap-southeast-2, ap-northeast-1, ap-northeast-2,
-  sa-east-1, cn-north-1, cn-northwest-1, ca-central-1
-  at org.apache.hadoop.fs.s3a.s3guard.DynamoDBClientFactory$DefaultDynamoDBClientFactory.getRegion
-  at org.apache.hadoop.fs.s3a.s3guard.DynamoDBClientFactory$DefaultDynamoDBClientFactory.createDynamoDBClient
-```
-
-The region specified in `fs.s3a.s3guard.ddb.region` is invalid.
 
 ## Other Topics
 

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

@@ -52,15 +52,12 @@ import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.util.StringUtils;
 
 import static org.apache.hadoop.fs.s3a.Constants.METADATASTORE_AUTHORITATIVE;
-import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_REGION_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_CREATE_KEY;
 import static org.apache.hadoop.fs.s3a.Constants.S3GUARD_DDB_TABLE_NAME_KEY;
 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.E_NO_METASTORE_OR_FILESYSTEM;
-import static org.apache.hadoop.fs.s3a.s3guard.S3GuardTool.E_USAGE;
 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;
@@ -349,68 +346,28 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
   @Test
   public void testSetCapacityFailFastIfNotGuarded() throws Exception{
     Configuration conf = getConfiguration();
-    bindToNonexistentTable(conf);
-    String bucket = rawFs.getBucket();
-    clearBucketOption(conf, bucket, S3_METADATA_STORE_IMPL);
-    clearBucketOption(conf, bucket, S3GUARD_DDB_TABLE_NAME_KEY);
-    clearBucketOption(conf, bucket, S3GUARD_DDB_TABLE_CREATE_KEY);
+    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, UUID.randomUUID().toString());
+    conf.set(S3GUARD_DDB_TABLE_CREATE_KEY, Boolean.FALSE.toString());
     conf.set(S3_METADATA_STORE_IMPL, S3GUARD_METASTORE_NULL);
 
     S3GuardTool.SetCapacity cmdR = new S3GuardTool.SetCapacity(conf);
-    String[] argsR = new String[]{
-        cmdR.getName(),
-        "s3a://" + getFileSystem().getBucket()
-    };
+    String[] argsR = new String[]{cmdR.getName(),
+        "s3a://" + getFileSystem().getBucket()};
 
     intercept(IllegalStateException.class, "unguarded",
-        () -> cmdR.run(argsR));
-  }
-
-  /**
-   * Binds the configuration to a nonexistent table.
-   * @param conf
-   */
-  protected void bindToNonexistentTable(final Configuration conf) {
-    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, UUID.randomUUID().toString());
-    conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, false);
+        () -> run(argsR));
   }
 
   @Test
   public void testDestroyNoBucket() throws Throwable {
-    describe("Destroy a bucket which doesn't exist");
-
-    Configuration conf = getConfiguration();
-    // set a table as a safety check in case the test goes wrong
-    // and deletes it.
-    bindToNonexistentTable(conf);
-
-    S3GuardTool.Destroy cmdR = new S3GuardTool.Destroy(conf);
-    String[] argsR = new String[]{
-        S3GuardTool.Destroy.NAME,
-        S3A_THIS_BUCKET_DOES_NOT_EXIST
-    };
     intercept(FileNotFoundException.class,
-        () -> cmdR.run(argsR));
-  }
-
-  @Test
-  public void testDestroyNoArgs() throws Throwable {
-    describe("Destroy a bucket which doesn't exist");
-
-    Configuration conf = getConfiguration();
-    // set a table as a safety check in case the test goes wrong
-    // and deletes it.
-    conf.set(S3GUARD_DDB_TABLE_NAME_KEY, UUID.randomUUID().toString());
-    conf.set(S3GUARD_DDB_REGION_KEY, "us-gov-west-1");
-    conf.setBoolean(S3GUARD_DDB_TABLE_CREATE_KEY, false);
-
-    S3GuardTool.Destroy cmdR = new S3GuardTool.Destroy(conf);
-
-    assertExitCode(E_USAGE,
-        intercept(ExitUtil.ExitException.class,
-            E_NO_METASTORE_OR_FILESYSTEM,
-            () -> cmdR.run(new String[]{})));
-
+        new Callable<Integer>() {
+          @Override
+          public Integer call() throws Exception {
+            return run(S3GuardTool.Destroy.NAME,
+                S3A_THIS_BUCKET_DOES_NOT_EXIST);
+          }
+        });
   }
 
   @Test
@@ -425,24 +382,11 @@ public abstract class AbstractS3GuardToolTestBase extends AbstractS3ATestBase {
       exec(cmd, S3GuardTool.BucketInfo.MAGIC_FLAG, name);
     } else {
       // if the FS isn't magic, expect the probe to fail
-      assertExitCode(E_BAD_STATE,
-          intercept(ExitUtil.ExitException.class,
-              () -> exec(cmd, S3GuardTool.BucketInfo.MAGIC_FLAG, name)));
-    }
-  }
-
-  /**
-   * Assert that an exit exception had a specific error code.
-   * @param expectedErrorCode expected code.
-   * @param e exit exception
-   * @throws AssertionError with the exit exception nested inside
-   */
-  protected void assertExitCode(final int expectedErrorCode,
-      final ExitUtil.ExitException e) {
-    if (e.getExitCode() != expectedErrorCode) {
-      throw new AssertionError("Expected error code " + expectedErrorCode
-          + " in " + e,
-          e);
+      ExitUtil.ExitException e = intercept(ExitUtil.ExitException.class,
+          () -> exec(cmd, S3GuardTool.BucketInfo.MAGIC_FLAG, name));
+      if (e.getExitCode() != E_BAD_STATE) {
+        throw e;
+      }
     }
   }