瀏覽代碼

HDDS-539. Ozone datanode command ignores the invalid options. Contributed by Vinicius Higa Murakami.

Shashikant Banerjee 6 年之前
父節點
當前提交
ee10ba26dd

+ 69 - 49
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java

@@ -1,19 +1,19 @@
-/*
+/**
  * 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.
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone;
 
@@ -21,20 +21,20 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.HddsUtils;
+import org.apache.hadoop.hdds.cli.GenericCli;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
-import org.apache.hadoop.ozone.container.common.statemachine
-    .DatanodeStateMachine;
-import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
 import org.apache.hadoop.util.ServicePlugin;
 import org.apache.hadoop.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
 
 import java.io.File;
 import java.io.IOException;
@@ -48,12 +48,16 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
 /**
  * Datanode service plugin to start the HDDS container services.
  */
-public class HddsDatanodeService implements ServicePlugin {
+
+@Command(name = "ozone datanode",
+    hidden = true, description = "Start the datanode for ozone",
+    versionProvider = HddsVersionProvider.class,
+    mixinStandardHelpOptions = true)
+public class HddsDatanodeService extends GenericCli implements ServicePlugin {
 
   private static final Logger LOG = LoggerFactory.getLogger(
       HddsDatanodeService.class);
 
-
   private OzoneConfiguration conf;
   private DatanodeDetails datanodeDetails;
   private DatanodeStateMachine datanodeStateMachine;
@@ -80,6 +84,50 @@ public class HddsDatanodeService implements ServicePlugin {
     }
   }
 
+  @VisibleForTesting
+  public static HddsDatanodeService createHddsDatanodeService(
+      String[] args, Configuration conf) {
+    return createHddsDatanodeService(args, conf, true);
+  }
+
+  /**
+   * 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);
+    } else {
+      new HddsDatanodeService().run(args);
+      return null;
+    }
+  }
+
+  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();
+      }
+    } catch (Throwable e) {
+      LOG.error("Exception in HddsDatanodeService.", e);
+      terminate(1, e);
+    }
+  }
+
   /**
    * Starts HddsDatanode services.
    *
@@ -87,6 +135,7 @@ public class HddsDatanodeService implements ServicePlugin {
    */
   @Override
   public void start(Object service) {
+    DefaultMetricsSystem.initialize("HddsDatanode");
     OzoneConfiguration.activate();
     if (service instanceof Configurable) {
       conf = new OzoneConfiguration(((Configurable) service).getConf());
@@ -148,7 +197,7 @@ public class HddsDatanodeService implements ServicePlugin {
     } catch (RuntimeException e) {
       String pluginsValue = conf.get(HDDS_DATANODE_PLUGINS_KEY);
       LOG.error("Unable to load HDDS DataNode plugins. " +
-          "Specified list of plugins: {}",
+              "Specified list of plugins: {}",
           pluginsValue, e);
       throw e;
     }
@@ -170,8 +219,8 @@ public class HddsDatanodeService implements ServicePlugin {
   public OzoneConfiguration getConf() {
     return conf;
   }
+
   /**
-   *
    * Return DatanodeDetails if set, return null otherwise.
    *
    * @return DatanodeDetails
@@ -213,7 +262,7 @@ public class HddsDatanodeService implements ServicePlugin {
   }
 
   @Override
-  public void close() throws IOException {
+  public void close() {
     if (plugins != null) {
       for (ServicePlugin plugin : plugins) {
         try {
@@ -224,33 +273,4 @@ public class HddsDatanodeService implements ServicePlugin {
       }
     }
   }
-
-  public static HddsDatanodeService createHddsDatanodeService(
-      Configuration conf) {
-    return new HddsDatanodeService(conf);
-  }
-
-  public static void main(String[] args) {
-    try {
-      if (DFSUtil.parseHelpArgument(
-          args, "Starts HDDS Datanode", System.out, false)) {
-        System.exit(0);
-      }
-      Configuration conf = new OzoneConfiguration();
-      GenericOptionsParser hParser = new GenericOptionsParser(conf, args);
-      if (!hParser.isParseSuccessful()) {
-        GenericOptionsParser.printGenericCommandUsage(System.err);
-        System.exit(1);
-      }
-      StringUtils.startupShutdownMessage(HddsDatanodeService.class, args, LOG);
-      DefaultMetricsSystem.initialize("HddsDatanode");
-      HddsDatanodeService hddsDatanodeService =
-          createHddsDatanodeService(conf);
-      hddsDatanodeService.start(null);
-      hddsDatanodeService.join();
-    } catch (Throwable e) {
-      LOG.error("Exception in HddsDatanodeService.", e);
-      terminate(1, e);
-    }
-  }
 }

+ 2 - 1
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/TestHddsDatanodeService.java

@@ -42,6 +42,7 @@ public class TestHddsDatanodeService {
   private File testDir;
   private Configuration conf;
   private HddsDatanodeService service;
+  private String[] args = new String[] {};
 
   @Before
   public void setUp() {
@@ -63,7 +64,7 @@ public class TestHddsDatanodeService {
 
   @Test
   public void testStartup() throws IOException {
-    service = HddsDatanodeService.createHddsDatanodeService(conf);
+    service = HddsDatanodeService.createHddsDatanodeService(args, conf);
     service.start(null);
     service.join();
 

+ 4 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java

@@ -268,8 +268,9 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
       // wait for node to be removed from SCM healthy node list.
       waitForClusterToBeReady();
     }
+    String[] args = new String[]{};
     HddsDatanodeService service =
-        HddsDatanodeService.createHddsDatanodeService(conf);
+        HddsDatanodeService.createHddsDatanodeService(args, conf);
     hddsDatanodes.add(i, service);
     service.start(null);
     if (waitForDatanode) {
@@ -478,6 +479,7 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
       configureHddsDatanodes();
       String scmAddress =  scm.getDatanodeRpcAddress().getHostString() +
           ":" + scm.getDatanodeRpcAddress().getPort();
+      String[] args = new String[] {};
       conf.setStrings(ScmConfigKeys.OZONE_SCM_NAMES, scmAddress);
       List<HddsDatanodeService> hddsDatanodes = new ArrayList<>();
       for (int i = 0; i < numOfDatanodes; i++) {
@@ -495,7 +497,7 @@ public final class MiniOzoneClusterImpl implements MiniOzoneCluster {
             ratisDir.toString());
 
         hddsDatanodes.add(
-            HddsDatanodeService.createHddsDatanodeService(dnConf));
+            HddsDatanodeService.createHddsDatanodeService(args, dnConf));
       }
       return hddsDatanodes;
     }

+ 250 - 0
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneDatanodeShell.java

@@ -0,0 +1,250 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.ozShell;
+
+import com.google.common.base.Strings;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+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.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.HddsDatanodeService;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rest.RestClient;
+import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+import picocli.CommandLine.ExecutionException;
+import picocli.CommandLine.IExceptionHandler2;
+import picocli.CommandLine.ParameterException;
+import picocli.CommandLine.ParseResult;
+import picocli.CommandLine.RunLast;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.junit.Assert.fail;
+
+/**
+ * This test class specified for testing Ozone datanode shell command.
+ */
+@RunWith(value = Parameterized.class)
+
+public class TestOzoneDatanodeShell {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOzoneDatanodeShell.class);
+
+  /**
+   * Set the timeout for every test.
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(300000);
+
+  private static String url;
+  private static File baseDir;
+  private static OzoneConfiguration conf = null;
+  private static MiniOzoneCluster cluster = null;
+  private static ClientProtocol client = null;
+  private static HddsDatanodeService datanode = null;
+
+  private final ByteArrayOutputStream out = new ByteArrayOutputStream();
+  private final ByteArrayOutputStream err = new ByteArrayOutputStream();
+  private static final PrintStream OLD_OUT = System.out;
+  private static final PrintStream OLD_ERR = System.err;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> clientProtocol() {
+    Object[][] params = new Object[][]{
+        {RpcClient.class},
+        {RestClient.class}};
+    return Arrays.asList(params);
+  }
+
+  @Parameterized.Parameter
+  public Class clientProtocol;
+  /**
+   * Create a MiniDFSCluster for testing with using distributed Ozone
+   * handler type.
+   *
+   * @throws Exception
+   */
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new OzoneConfiguration();
+
+    String path = GenericTestUtils.getTempPath(
+        TestOzoneShell.class.getSimpleName());
+    baseDir = new File(path);
+    baseDir.mkdirs();
+
+    datanode = new HddsDatanodeService();
+
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(3)
+        .build();
+    conf.setInt(OZONE_REPLICATION, ReplicationFactor.THREE.getValue());
+    conf.setQuietMode(false);
+    client = new RpcClient(conf);
+    cluster.waitForClusterToBeReady();
+  }
+
+  /**
+   * shutdown MiniDFSCluster.
+   */
+  @AfterClass
+  public static void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+
+    if (baseDir != null) {
+      FileUtil.fullyDelete(baseDir, true);
+    }
+  }
+
+  @Before
+  public void setup() {
+    System.setOut(new PrintStream(out));
+    System.setErr(new PrintStream(err));
+    if(clientProtocol.equals(RestClient.class)) {
+      String hostName = cluster.getOzoneManager().getHttpServer()
+          .getHttpAddress().getHostName();
+      int port = cluster
+          .getOzoneManager().getHttpServer().getHttpAddress().getPort();
+      url = String.format("http://" + hostName + ":" + port);
+    } else {
+      List<ServiceInfo> services = null;
+      try {
+        services = cluster.getOzoneManager().getServiceList();
+      } catch (IOException e) {
+        LOG.error("Could not get service list from OM");
+      }
+      String hostName = services.stream().filter(
+          a -> a.getNodeType().equals(HddsProtos.NodeType.OM))
+          .collect(Collectors.toList()).get(0).getHostname();
+
+      String port = cluster.getOzoneManager().getRpcPort();
+      url = String.format("o3://" + hostName + ":" + port);
+    }
+  }
+
+  @After
+  public void reset() {
+    // reset stream after each unit test
+    out.reset();
+    err.reset();
+
+    // restore system streams
+    System.setOut(OLD_OUT);
+    System.setErr(OLD_ERR);
+  }
+
+
+  private void executeDatanode(HddsDatanodeService hdds, String[] args) {
+    List<String> arguments = new ArrayList(Arrays.asList(args));
+    LOG.info("Executing ozone datanode command with args {}", arguments);
+    CommandLine cmd = datanode.getCmd();
+
+    IExceptionHandler2<List<Object>> exceptionHandler =
+        new IExceptionHandler2<List<Object>>() {
+          @Override
+          public List<Object> handleParseException(ParameterException ex,
+              String[] args) {
+            throw ex;
+          }
+
+          @Override
+          public List<Object> handleExecutionException(ExecutionException ex,
+              ParseResult parseResult) {
+            throw ex;
+          }
+        };
+    cmd.parseWithHandlers(new RunLast(),
+        exceptionHandler, args);
+  }
+
+  /**
+   * Execute command, assert exception message and returns true if error
+   * was thrown and contains the specified usage string.
+   */
+  private void executeDatanodeWithError(HddsDatanodeService hdds, String[] args,
+      String expectedError, String usage) {
+    if (Strings.isNullOrEmpty(expectedError)) {
+      executeDatanode(hdds, args);
+    } else {
+      try {
+        executeDatanode(hdds, args);
+        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));
+        }
+      }
+    }
+  }
+
+  @Test
+  public void testDatanodeIncompleteCommand() {
+    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>]...");
+
+  }
+}