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 el...@apache.org on 2019/04/24 13:35:22 UTC

[hadoop] branch trunk updated: HDDS-1192. Support -conf command line argument in GenericCli

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

elek 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 3f787cd  HDDS-1192. Support -conf command line argument in GenericCli
3f787cd is described below

commit 3f787cd5065560f1bbb9f56a617cd4815803ca8a
Author: Kitti Nanasi <kn...@cloudera.com>
AuthorDate: Wed Apr 24 15:07:13 2019 +0200

    HDDS-1192. Support -conf command line argument in GenericCli
    
    Closes #713
---
 .../org/apache/hadoop/hdds/cli/GenericCli.java     | 12 ++-
 .../hdds/cli/MissingSubcommandException.java       | 14 ++--
 .../apache/hadoop/ozone/HddsDatanodeService.java   | 95 ++++++++++++----------
 .../hadoop/ozone/TestHddsDatanodeService.java      |  7 +-
 .../hadoop/ozone/TestHddsSecureDatanodeInit.java   |  4 +-
 .../hadoop/hdds/scm/cli/SafeModeCommands.java      |  3 +-
 .../apache/hadoop/ozone/MiniOzoneClusterImpl.java  | 23 +++---
 .../apache/hadoop/ozone/om/TestScmSafeMode.java    |  2 +-
 .../ozone/ozShell/TestOzoneDatanodeShell.java      | 26 +++---
 .../hadoop/ozone/ozShell/TestOzoneShell.java       | 69 ++++++++--------
 .../ozone/web/ozShell/bucket/BucketCommands.java   |  2 +-
 .../hadoop/ozone/web/ozShell/keys/KeyCommands.java |  2 +-
 .../ozone/web/ozShell/token/TokenCommands.java     |  2 +-
 .../ozone/web/ozShell/volume/VolumeCommands.java   |  2 +-
 .../hadoop/ozone/audit/parser/TestAuditParser.java |  7 +-
 15 files changed, 138 insertions(+), 132 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
index f905f60..372828b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
@@ -22,6 +22,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.Callable;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.fs.Path;
 
 import com.google.common.annotations.VisibleForTesting;
 import picocli.CommandLine;
@@ -41,6 +42,9 @@ public class GenericCli implements Callable<Void>, GenericParentCommand {
   @Option(names = {"-D", "--set"})
   private Map<String, String> configurationOverrides = new HashMap<>();
 
+  @Option(names = {"-conf"})
+  private String configurationPath;
+
   private final CommandLine cmd;
 
   public GenericCli() {
@@ -70,19 +74,19 @@ public class GenericCli implements Callable<Void>, GenericParentCommand {
     } else {
       System.err.println(error.getMessage().split("\n")[0]);
     }
-    if(error instanceof MissingSubcommandException){
-      System.err.println(((MissingSubcommandException) error).getUsage());
-    }
   }
 
   @Override
   public Void call() throws Exception {
-    throw new MissingSubcommandException(cmd.getUsageMessage());
+    throw new MissingSubcommandException(cmd);
   }
 
   @Override
   public OzoneConfiguration createOzoneConfiguration() {
     OzoneConfiguration ozoneConf = new OzoneConfiguration();
+    if (configurationPath != null) {
+      ozoneConf.addResource(new Path(configurationPath));
+    }
     if (configurationOverrides != null) {
       for (Entry<String, String> entry : configurationOverrides.entrySet()) {
         ozoneConf.set(entry.getKey(), entry.getValue());
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/MissingSubcommandException.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/MissingSubcommandException.java
index 9f0c494..7594765 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/MissingSubcommandException.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/MissingSubcommandException.java
@@ -17,19 +17,15 @@
  */
 package org.apache.hadoop.hdds.cli;
 
+import picocli.CommandLine;
+
 /**
  * Exception to throw if subcommand is not selected but required.
  */
-public class MissingSubcommandException extends RuntimeException {
-
-  private String usage;
+public class MissingSubcommandException extends CommandLine.ParameterException {
 
-  public MissingSubcommandException(String usage) {
-    super("Incomplete command");
-    this.usage = usage;
+  public MissingSubcommandException(CommandLine cmd) {
+    super(cmd, "Incomplete command");
   }
 
-  public String getUsage() {
-    return usage;
-  }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
index 5228335..df2e167 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java
@@ -53,6 +53,7 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.security.KeyPair;
 import java.security.cert.CertificateException;
+import java.util.Arrays;
 import java.util.List;
 import java.util.UUID;
 
@@ -81,64 +82,46 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
   private CertificateClient dnCertClient;
   private String component;
   private HddsDatanodeHttpServer httpServer;
+  private boolean printBanner;
+  private String[] args;
 
-  /**
-   * Default constructor.
-   */
-  public HddsDatanodeService() {
-    this(null);
+  public HddsDatanodeService(boolean printBanner, String[] args) {
+    this.printBanner = printBanner;
+    this.args = args != null ? Arrays.copyOf(args, args.length) : null;
   }
 
   /**
-   * Constructs {@link HddsDatanodeService} using the provided {@code conf}
-   * value.
+   * Create an Datanode instance based on the supplied command-line arguments.
+   * <p>
+   * This method is intended for unit tests only. It suppresses the
+   * startup/shutdown message and skips registering Unix signal handlers.
    *
-   * @param conf OzoneConfiguration
+   * @param args      command line arguments.
+   * @return Datanode instance
    */
-  public HddsDatanodeService(Configuration conf) {
-    if (conf == null) {
-      this.conf = new OzoneConfiguration();
-    } else {
-      this.conf = new OzoneConfiguration(conf);
-    }
-  }
-
   @VisibleForTesting
   public static HddsDatanodeService createHddsDatanodeService(
-      String[] args, Configuration conf) {
-    return createHddsDatanodeService(args, conf, false);
+      String[] args) {
+    return createHddsDatanodeService(args, false);
   }
 
   /**
    * Create an Datanode instance based on the supplied command-line arguments.
-   * <p>
-   * This method is intended for unit tests only. It suppresses the
-   * startup/shutdown message and skips registering Unix signal handlers.
    *
    * @param args        command line arguments.
-   * @param conf        HDDS configuration
    * @param printBanner if true, then log a verbose startup message.
    * @return Datanode instance
    */
   private static HddsDatanodeService createHddsDatanodeService(
-      String[] args, Configuration conf, boolean printBanner) {
-    if (args.length == 0 && printBanner) {
-      StringUtils
-          .startupShutdownMessage(HddsDatanodeService.class, args, LOG);
-
-    }
-    return new HddsDatanodeService(conf);
+      String[] args, boolean printBanner) {
+    return new HddsDatanodeService(printBanner, args);
   }
 
   public static void main(String[] args) {
     try {
-      Configuration conf = new OzoneConfiguration();
       HddsDatanodeService hddsDatanodeService =
-          createHddsDatanodeService(args, conf, true);
-      if (hddsDatanodeService != null) {
-        hddsDatanodeService.start(null);
-        hddsDatanodeService.join();
-      }
+          createHddsDatanodeService(args, true);
+      hddsDatanodeService.run(args);
     } catch (Throwable e) {
       LOG.error("Exception in HddsDatanodeService.", e);
       terminate(1, e);
@@ -149,6 +132,21 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
     return LOG;
   }
 
+  @Override
+  public Void call() throws Exception {
+    if (printBanner) {
+      StringUtils
+          .startupShutdownMessage(HddsDatanodeService.class, args, LOG);
+    }
+    start(createOzoneConfiguration());
+    join();
+    return null;
+  }
+
+  public void setConfiguration(OzoneConfiguration configuration) {
+    this.conf = configuration;
+  }
+
   /**
    * Starts HddsDatanode services.
    *
@@ -156,12 +154,21 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
    */
   @Override
   public void start(Object service) {
+    if (service instanceof Configurable) {
+      start(new OzoneConfiguration(((Configurable) service).getConf()));
+    } else {
+      start(new OzoneConfiguration());
+    }
+  }
+
+  public void start(OzoneConfiguration configuration) {
+    setConfiguration(configuration);
+    start();
+  }
 
+  public void start() {
     DefaultMetricsSystem.initialize("HddsDatanode");
     OzoneConfiguration.activate();
-    if (service instanceof Configurable) {
-      conf = new OzoneConfiguration(((Configurable) service).getConf());
-    }
     if (HddsUtils.isHddsEnabled(conf)) {
       try {
         String hostname = HddsUtils.getHostName(conf);
@@ -404,11 +411,13 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
   }
 
   public void join() {
-    try {
-      datanodeStateMachine.join();
-    } catch (InterruptedException e) {
-      Thread.currentThread().interrupt();
-      LOG.info("Interrupted during StorageContainerManager join.");
+    if (datanodeStateMachine != null) {
+      try {
+        datanodeStateMachine.join();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        LOG.info("Interrupted during StorageContainerManager join.");
+      }
     }
   }
 
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/TestHddsDatanodeService.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/TestHddsDatanodeService.java
index b1ac4b6..f54fa75 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/TestHddsDatanodeService.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/TestHddsDatanodeService.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.assertNotNull;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -40,7 +39,7 @@ import org.junit.Test;
  */
 public class TestHddsDatanodeService {
   private File testDir;
-  private Configuration conf;
+  private OzoneConfiguration conf;
   private HddsDatanodeService service;
   private String[] args = new String[] {};
 
@@ -64,8 +63,8 @@ public class TestHddsDatanodeService {
 
   @Test
   public void testStartup() throws IOException {
-    service = HddsDatanodeService.createHddsDatanodeService(args, conf);
-    service.start(null);
+    service = HddsDatanodeService.createHddsDatanodeService(args);
+    service.start(conf);
     service.join();
 
     assertNotNull(service.getDatanodeDetails());
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/TestHddsSecureDatanodeInit.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/TestHddsSecureDatanodeInit.java
index 2897abc..20d5eef 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/TestHddsSecureDatanodeInit.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/TestHddsSecureDatanodeInit.java
@@ -83,10 +83,10 @@ public class TestHddsSecureDatanodeInit {
         ServicePlugin.class);
     securityConfig = new SecurityConfig(conf);
 
-    service = HddsDatanodeService.createHddsDatanodeService(args, conf);
+    service = HddsDatanodeService.createHddsDatanodeService(args);
     dnLogs = GenericTestUtils.LogCapturer.captureLogs(getLogger());
     callQuietly(() -> {
-      service.start(null);
+      service.start(conf);
       return null;
     });
     callQuietly(() -> {
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCommands.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCommands.java
index c1e953d..3a9a63c 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCommands.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCommands.java
@@ -54,7 +54,6 @@ public class SafeModeCommands implements Callable<Void> {
   @Override
   public Void call() throws Exception {
     throw new MissingSubcommandException(
-        this.parent.getCmd().getSubcommands().get("safemode").
-        getUsageMessage());
+        this.parent.getCmd().getSubcommands().get("safemode"));
   }
 }
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
index 8e28670..9fbdad7 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Optional;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@@ -104,9 +103,9 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
    * @throws IOException if there is an I/O error
    */
   MiniOzoneClusterImpl(OzoneConfiguration conf,
-                               OzoneManager ozoneManager,
-                               StorageContainerManager scm,
-                               List<HddsDatanodeService> hddsDatanodes) {
+                       OzoneManager ozoneManager,
+                       StorageContainerManager scm,
+                       List<HddsDatanodeService> hddsDatanodes) {
     this.conf = conf;
     this.ozoneManager = ozoneManager;
     this.scm = scm;
@@ -275,7 +274,7 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
     datanodeService.stop();
     datanodeService.join();
     // ensure same ports are used across restarts.
-    Configuration config = datanodeService.getConf();
+    OzoneConfiguration config = datanodeService.getConf();
     int currentPort = datanodeService.getDatanodeDetails()
         .getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
     config.setInt(DFS_CONTAINER_IPC_PORT, currentPort);
@@ -291,9 +290,9 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
     }
     String[] args = new String[]{};
     HddsDatanodeService service =
-        HddsDatanodeService.createHddsDatanodeService(args, config);
+        HddsDatanodeService.createHddsDatanodeService(args);
     hddsDatanodes.add(i, service);
-    service.start(null);
+    service.start(config);
     if (waitForDatanode) {
       // wait for the node to be identified as a healthy node again.
       waitForClusterToBeReady();
@@ -371,7 +370,7 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
   public void startHddsDatanodes() {
     hddsDatanodes.forEach((datanode) -> {
       datanode.setCertificateClient(getCAClient());
-      datanode.start(null);
+      datanode.start();
     });
   }
 
@@ -537,7 +536,7 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
       conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, scmAddress);
       List<HddsDatanodeService> hddsDatanodes = new ArrayList<>();
       for (int i = 0; i < numOfDatanodes; i++) {
-        Configuration dnConf = new OzoneConfiguration(conf);
+        OzoneConfiguration dnConf = new OzoneConfiguration(conf);
         String datanodeBaseDir = path + "/datanode-" + Integer.toString(i);
         Path metaDir = Paths.get(datanodeBaseDir, "meta");
         Path dataDir = Paths.get(datanodeBaseDir, "data", "containers");
@@ -555,8 +554,10 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
         dnConf.set(OzoneConfigKeys.OZONE_CONTAINER_COPY_WORKDIR,
             wrokDir.toString());
 
-        hddsDatanodes.add(
-            HddsDatanodeService.createHddsDatanodeService(args, dnConf));
+        HddsDatanodeService datanode
+            = HddsDatanodeService.createHddsDatanodeService(args);
+        datanode.setConfiguration(dnConf);
+        hddsDatanodes.add(datanode);
       }
       return hddsDatanodes;
     }
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java
index b34ea72..59331c9 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java
@@ -270,7 +270,7 @@ public class TestScmSafeMode {
     assertFalse(logCapturer.getOutput().contains("SCM exiting safe mode."));
     assertTrue(scm.getCurrentContainerThreshold() == 0);
     for (HddsDatanodeService dn : cluster.getHddsDatanodes()) {
-      dn.start(null);
+      dn.start();
     }
     GenericTestUtils
         .waitFor(() -> scm.getCurrentContainerThreshold() == 1.0, 100, 20000);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneDatanodeShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneDatanodeShell.java
index 9ba8529..e2942a4 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneDatanodeShell.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneDatanodeShell.java
@@ -27,7 +27,6 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdds.cli.MissingSubcommandException;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.HddsDatanodeService;
@@ -108,7 +107,7 @@ public class TestOzoneDatanodeShell {
     baseDir = new File(path);
     baseDir.mkdirs();
 
-    datanode = new HddsDatanodeService();
+    datanode = HddsDatanodeService.createHddsDatanodeService(null);
 
     cluster = MiniOzoneCluster.newBuilder(conf)
         .setNumDatanodes(3)
@@ -177,7 +176,7 @@ public class TestOzoneDatanodeShell {
    * was thrown and contains the specified usage string.
    */
   private void executeDatanodeWithError(HddsDatanodeService hdds, String[] args,
-      String expectedError, String usage) {
+      String expectedError) {
     if (Strings.isNullOrEmpty(expectedError)) {
       executeDatanode(hdds, args);
     } else {
@@ -197,24 +196,27 @@ public class TestOzoneDatanodeShell {
                       "exception [%s] in [%s]",
                   expectedError, exceptionToCheck.getMessage()),
               exceptionToCheck.getMessage().contains(expectedError));
-          Assert.assertTrue(
-              exceptionToCheck instanceof MissingSubcommandException);
-          Assert.assertTrue(
-              ((MissingSubcommandException) exceptionToCheck)
-                  .getUsage().contains(usage));
         }
       }
     }
   }
 
   @Test
-  public void testDatanodeIncompleteCommand() {
+  public void testDatanodeCommand() {
     LOG.info("Running testDatanodeIncompleteCommand");
-    String expectedError = "Incomplete command";
     String[] args = new String[]{}; //executing 'ozone datanode'
 
-    executeDatanodeWithError(datanode, args, expectedError,
-        "Usage: ozone datanode [-hV] [--verbose] [-D=<String=String>]...");
+    //'ozone datanode' command should not result in error
+    executeDatanodeWithError(datanode, args, null);
+  }
+
+  @Test
+  public void testDatanodeInvalidParamCommand() {
+    LOG.info("Running testDatanodeIncompleteCommand");
+    String expectedError = "Unknown option: -invalidParam";
+    //executing 'ozone datanode -invalidParam'
+    String[] args = new String[]{"-invalidParam"};
 
+    executeDatanodeWithError(datanode, args, expectedError);
   }
 }
\ No newline at end of file
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
index 0b53f69..d3a19c3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
@@ -361,21 +361,19 @@ public class TestOzoneShell {
     String expectedError = "Incomplete command";
     String[] args = new String[] {}; //executing 'ozone sh'
 
-    executeWithError(shell, args, expectedError,
-        "Usage: ozone sh [-hV] [--verbose] [-D=<String=String>]..." +
-            " [COMMAND]");
+    executeWithError(shell, args, expectedError);
 
     args = new String[] {"volume"}; //executing 'ozone sh volume'
-    executeWithError(shell, args, expectedError,
-        "Usage: ozone sh volume [-hV] [COMMAND]");
+    executeWithError(shell, args, MissingSubcommandException.class,
+        expectedError);
 
     args = new String[] {"bucket"}; //executing 'ozone sh bucket'
-    executeWithError(shell, args, expectedError,
-        "Usage: ozone sh bucket [-hV] [COMMAND]");
+    executeWithError(shell, args, MissingSubcommandException.class,
+        expectedError);
 
     args = new String[] {"key"}; //executing 'ozone sh key'
-    executeWithError(shell, args, expectedError,
-        "Usage: ozone sh key [-hV] [COMMAND]");
+    executeWithError(shell, args, MissingSubcommandException.class,
+        expectedError);
   }
 
   @Test
@@ -475,12 +473,12 @@ public class TestOzoneShell {
   }
 
   /**
-   * Execute command, assert exception message and returns true if error
-   * was thrown.
+   * Execute command, assert exception message and exception class
+   * and returns true if error was thrown.
    */
   private void executeWithError(Shell ozoneShell, String[] args,
-      Class exception) {
-    if (Objects.isNull(exception)) {
+                                Class expectedException, String expectedError) {
+    if (Strings.isNullOrEmpty(expectedError)) {
       execute(ozoneShell, args);
     } else {
       try {
@@ -488,20 +486,31 @@ public class TestOzoneShell {
         fail("Exception is expected from command execution " + Arrays
             .asList(args));
       } catch (Exception ex) {
-        LOG.error("Exception: ", ex);
-        assertTrue(ex.getCause().getClass().getCanonicalName()
-            .equals(exception.getCanonicalName()));
+        if (!Strings.isNullOrEmpty(expectedError)) {
+          Throwable exceptionToCheck = ex;
+          if (exceptionToCheck.getCause() != null) {
+            exceptionToCheck = exceptionToCheck.getCause();
+          }
+          Assert.assertTrue(
+              String.format(
+                  "Error of shell code doesn't contain the " +
+                      "expectedException [%s] in [%s]",
+                  expectedError, exceptionToCheck.getMessage()),
+              exceptionToCheck.getMessage().contains(expectedError));
+          assertTrue(ex.getClass().getCanonicalName()
+              .equals(expectedException.getCanonicalName()));
+        }
       }
     }
   }
 
   /**
-   * Execute command, assert exception message and returns true if error
-   * was thrown and contains the specified usage string.
+   * Execute command, assert exception cause message and returns true if error
+   * was thrown.
    */
   private void executeWithError(Shell ozoneShell, String[] args,
-      String expectedError, String usage) {
-    if (Strings.isNullOrEmpty(expectedError)) {
+      Class expectedCause) {
+    if (Objects.isNull(expectedCause)) {
       execute(ozoneShell, args);
     } else {
       try {
@@ -509,23 +518,9 @@ public class TestOzoneShell {
         fail("Exception is expected from command execution " + Arrays
             .asList(args));
       } catch (Exception ex) {
-        if (!Strings.isNullOrEmpty(expectedError)) {
-          Throwable exceptionToCheck = ex;
-          if (exceptionToCheck.getCause() != null) {
-            exceptionToCheck = exceptionToCheck.getCause();
-          }
-          Assert.assertTrue(
-              String.format(
-                  "Error of shell code doesn't contain the " +
-                      "exception [%s] in [%s]",
-                  expectedError, exceptionToCheck.getMessage()),
-              exceptionToCheck.getMessage().contains(expectedError));
-          Assert.assertTrue(
-              exceptionToCheck instanceof MissingSubcommandException);
-          Assert.assertTrue(
-              ((MissingSubcommandException)exceptionToCheck)
-                  .getUsage().contains(usage));
-        }
+        LOG.error("Exception: ", ex);
+        assertTrue(ex.getCause().getClass().getCanonicalName()
+            .equals(expectedCause.getCanonicalName()));
       }
     }
   }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/BucketCommands.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/BucketCommands.java
index 525e07f..64dc91b 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/BucketCommands.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/bucket/BucketCommands.java
@@ -51,7 +51,7 @@ public class BucketCommands implements GenericParentCommand, Callable<Void> {
   @Override
   public Void call() throws Exception {
     throw new MissingSubcommandException(
-        this.shell.getCmd().getSubcommands().get("bucket").getUsageMessage());
+        this.shell.getCmd().getSubcommands().get("bucket"));
   }
 
   @Override
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/KeyCommands.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/KeyCommands.java
index 0a76c54..405c3c5 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/KeyCommands.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/keys/KeyCommands.java
@@ -52,7 +52,7 @@ public class KeyCommands implements GenericParentCommand, Callable<Void> {
   @Override
   public Void call() throws Exception {
     throw new MissingSubcommandException(
-        this.shell.getCmd().getSubcommands().get("key").getUsageMessage());
+        this.shell.getCmd().getSubcommands().get("key"));
   }
 
   @Override
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/TokenCommands.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/TokenCommands.java
index fd4388d..2501ad9 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/TokenCommands.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/token/TokenCommands.java
@@ -49,7 +49,7 @@ public class TokenCommands implements GenericParentCommand, Callable<Void> {
   @Override
   public Void call() throws Exception {
     throw new MissingSubcommandException(
-        this.shell.getCmd().getSubcommands().get("token").getUsageMessage());
+        this.shell.getCmd().getSubcommands().get("token"));
   }
 
   @Override
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/VolumeCommands.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/VolumeCommands.java
index 1c3afe9..4fb71c3 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/VolumeCommands.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/volume/VolumeCommands.java
@@ -52,7 +52,7 @@ public class VolumeCommands implements GenericParentCommand, Callable<Void> {
   @Override
   public Void call() throws Exception {
     throw new MissingSubcommandException(
-        this.shell.getCmd().getSubcommands().get("volume").getUsageMessage());
+        this.shell.getCmd().getSubcommands().get("volume"));
   }
 
   @Override
diff --git a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/audit/parser/TestAuditParser.java b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/audit/parser/TestAuditParser.java
index ada56a5..900ea7c 100644
--- a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/audit/parser/TestAuditParser.java
+++ b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/audit/parser/TestAuditParser.java
@@ -177,9 +177,10 @@ public class TestAuditParser {
   public void testHelp() throws Exception {
     String[] args = new String[]{"--help"};
     execute(args,
-        "Usage: ozone auditparser [-hV] [--verbose] [-D=<String=String>]... " +
-            "<database>\n" +
-            "                         [COMMAND]");
+        "Usage: ozone auditparser [-hV] [--verbose] " +
+            "[-conf=<configurationPath>]\n" +
+            "                         [-D=<String=String>]... <database> " +
+            "[COMMAND]");
   }
 
   private static File getRandomTempDir() throws IOException {


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