ソースを参照

HDDS-1660. Use Picocli for Ozone Manager

Closes #925
S O'Donnell 5 年 前
コミット
22b36dd311

+ 1 - 1
hadoop-ozone/common/src/main/bin/ozone

@@ -129,7 +129,7 @@ function ozonecmd_case
     ;;
     om)
       HADOOP_SUBCMD_SUPPORTDAEMONIZATION="true"
-      HADOOP_CLASSNAME=org.apache.hadoop.ozone.om.OzoneManager
+      HADOOP_CLASSNAME=org.apache.hadoop.ozone.om.OzoneManagerStarter
       HDFS_OM_OPTS="${HDFS_OM_OPTS} -Dlog4j.configurationFile=${HADOOP_CONF_DIR}/om-audit-log4j2.properties"
       HADOOP_OPTS="${HADOOP_OPTS} ${HDFS_OM_OPTS}"
       OZONE_RUN_ARTIFACT_NAME="hadoop-ozone-ozone-manager"

+ 2 - 2
hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManager.xml

@@ -16,7 +16,7 @@
 -->
 <component name="ProjectRunConfigurationManager">
   <configuration default="false" name="OzoneManager" type="Application" factoryName="Application">
-    <option name="MAIN_CLASS_NAME" value="org.apache.hadoop.ozone.om.OzoneManager" />
+    <option name="MAIN_CLASS_NAME" value="org.apache.hadoop.ozone.om.OzoneManagerStarter" />
     <module name="hadoop-ozone-ozone-manager" />
     <option name="PROGRAM_PARAMETERS" value="-conf=hadoop-ozone/dev-support/intellij/ozone-site.xml" />
     <option name="VM_PARAMETERS" value="-Dlog4j.configuration=file:hadoop-ozone/dev-support/intellij/log4j.properties" />
@@ -30,4 +30,4 @@
       <option name="Make" enabled="true" />
     </method>
   </configuration>
-</component>
+</component>

+ 2 - 2
hadoop-ozone/dev-support/intellij/runConfigurations/OzoneManagerInit.xml

@@ -16,7 +16,7 @@
 -->
 <component name="ProjectRunConfigurationManager">
   <configuration default="false" name="OzoneManagerInit" type="Application" factoryName="Application">
-    <option name="MAIN_CLASS_NAME" value="org.apache.hadoop.ozone.om.OzoneManager" />
+    <option name="MAIN_CLASS_NAME" value="org.apache.hadoop.ozone.om.OzoneManagerStarter" />
     <module name="hadoop-ozone-ozone-manager" />
     <option name="PROGRAM_PARAMETERS" value="-conf=hadoop-ozone/dev-support/intellij/ozone-site.xml --init" />
     <option name="VM_PARAMETERS" value="-Dlog4j.configuration=file:hadoop-ozone/dev-support/intellij/log4j.properties" />
@@ -30,4 +30,4 @@
       <option name="Make" enabled="true" />
     </method>
   </configuration>
-</component>
+</component>

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

@@ -517,7 +517,7 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
       configureOM();
       OMStorage omStore = new OMStorage(conf);
       initializeOmStorage(omStore);
-      return OzoneManager.createOm(null, conf);
+      return OzoneManager.createOm(conf);
     }
 
     /**

+ 1 - 1
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneHAClusterImpl.java

@@ -211,7 +211,7 @@ public final class MiniOzoneHAClusterImpl extends MiniOzoneClusterImpl {
             OMStorage omStore = new OMStorage(conf);
             initializeOmStorage(omStore);
 
-            OzoneManager om = OzoneManager.createOm(null, conf);
+            OzoneManager om = OzoneManager.createOm(conf);
             om.setCertClient(certClient);
             omMap.put(nodeId, om);
 

+ 25 - 5
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java

@@ -383,7 +383,7 @@ public final class TestSecureOzoneCluster {
     setupOm(conf);
     conf.set(OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY,
         "non-existent-user@EXAMPLE.com");
-    testCommonKerberosFailures(() -> OzoneManager.createOm(null, conf));
+    testCommonKerberosFailures(() -> OzoneManager.createOm(conf));
   }
 
   /**
@@ -662,7 +662,7 @@ public final class TestSecureOzoneCluster {
     // writes the version file properties
     omStore.initialize();
     OzoneManager.setTestSecureOmFlag(true);
-    om = OzoneManager.createOm(null, config);
+    om = OzoneManager.createOm(config);
   }
 
   @Test
@@ -717,6 +717,26 @@ public final class TestSecureOzoneCluster {
     LogCapturer omLogs =
         LogCapturer.captureLogs(OzoneManager.getLogger());
     omLogs.clearOutput();
+
+    /**
+     * As all these processes run inside the same JVM, there are issues around
+     * the Hadoop UGI if different processes run with different principals.
+     * In this test, the OM has to contact the SCM to download certs. SCM runs
+     * as scm/host@REALM, but the OM logs in as om/host@REALM, and then the test
+     * fails, and the OM is unable to contact the SCM due to kerberos login
+     * issues. To work around that, have the OM run as the same principal as the
+     * SCM, and then the test passes.
+     *
+     * TODO: Need to look into this further to see if there is a better way to
+     *       address this problem.
+     */
+    String realm = miniKdc.getRealm();
+    conf.set(OMConfigKeys.OZONE_OM_KERBEROS_PRINCIPAL_KEY,
+        "scm/" + host + "@" + realm);
+    omKeyTab = new File(workDir, "scm.keytab");
+    conf.set(OMConfigKeys.OZONE_OM_KERBEROS_KEYTAB_FILE_KEY,
+        omKeyTab.getAbsolutePath());
+
     initSCM();
     try {
       scm = StorageContainerManager.createSCM(conf);
@@ -725,7 +745,7 @@ public final class TestSecureOzoneCluster {
       OMStorage omStore = new OMStorage(conf);
       initializeOmStorage(omStore);
       OzoneManager.setTestSecureOmFlag(true);
-      om = OzoneManager.createOm(null, conf);
+      om = OzoneManager.createOm(conf);
 
       assertNull(om.getCertificateClient());
       assertFalse(omLogs.getOutput().contains("Init response: GETCERT"));
@@ -735,7 +755,7 @@ public final class TestSecureOzoneCluster {
       conf.setBoolean(OZONE_SECURITY_ENABLED_KEY, true);
       OzoneManager.omInit(conf);
       om.stop();
-      om = OzoneManager.createOm(null, conf);
+      om = OzoneManager.createOm(conf);
 
       Assert.assertNotNull(om.getCertificateClient());
       Assert.assertNotNull(om.getCertificateClient().getPublicKey());
@@ -771,7 +791,7 @@ public final class TestSecureOzoneCluster {
       OMStorage omStore = new OMStorage(conf);
       initializeOmStorage(omStore);
       OzoneManager.setTestSecureOmFlag(true);
-      om = OzoneManager.createOm(null, conf);
+      om = OzoneManager.createOm(conf);
 
       Assert.assertNotNull(om.getCertificateClient());
       Assert.assertNotNull(om.getCertificateClient().getPublicKey());

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

@@ -23,6 +23,7 @@ import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -90,10 +91,11 @@ public class TestOmInit {
 
   /**
    * Tests the OM Initialization.
-   * @throws IOException
+   * @throws IOException, AuthenticationException
    */
   @Test
-  public void testOmInitAgain() throws IOException {
+  public void testOmInitAgain() throws IOException,
+      AuthenticationException {
     // Stop the Ozone Manager
     cluster.getOzoneManager().stop();
     // Now try to init the OM again. It should succeed

+ 2 - 2
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java

@@ -1326,7 +1326,7 @@ public class TestOzoneManager {
         conf.get(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY));
 
     OzoneTestUtils.expectOmException(ResultCodes.OM_NOT_INITIALIZED, () -> {
-      OzoneManager.createOm(null, config);
+      OzoneManager.createOm(config);
     });
 
     OzoneTestUtils
@@ -1336,7 +1336,7 @@ public class TestOzoneManager {
           omStore.setScmId("testScmId");
           // writes the version file properties
           omStore.initialize();
-          OzoneManager.createOm(null, config);
+          OzoneManager.createOm(config);
         });
   }
 

+ 33 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMStarterInterface.java

@@ -0,0 +1,33 @@
+/**
+ * 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.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import java.io.IOException;
+
+/**
+ * This interface is used by the OzoneManagerStarter class to allow the
+ * dependencies to be injected to the CLI class.
+ */
+public interface OMStarterInterface {
+  void start(OzoneConfiguration conf) throws IOException,
+      AuthenticationException;
+  boolean init(OzoneConfiguration conf) throws IOException,
+      AuthenticationException;
+}

+ 10 - 134
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java

@@ -142,12 +142,10 @@ import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.JvmPauseMonitor;
 import org.apache.hadoop.util.KMSUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.utils.RetriableTask;
 import org.apache.ratis.util.LifeCycle;
 import org.bouncycastle.pkcs.PKCS10CertificationRequest;
@@ -160,7 +158,6 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
-import java.io.PrintStream;
 import java.net.InetSocketAddress;
 import java.nio.file.Files;
 import java.nio.file.StandardCopyOption;
@@ -206,7 +203,6 @@ import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKE
 import static org.apache.hadoop.ozone.protocol.proto
     .OzoneManagerProtocolProtos.OzoneManagerService
     .newReflectiveBlockingService;
-import static org.apache.hadoop.util.ExitUtil.terminate;
 
 /**
  * Ozone Manager is the metadata manager of ozone.
@@ -220,10 +216,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
   private static final AuditLogger AUDIT = new AuditLogger(
       AuditLoggerType.OMLOGGER);
 
-  private static final String USAGE =
-      "Usage: \n ozone om [genericOptions] " + "[ "
-          + StartupOption.INIT.getName() + " ]\n " + "ozone om [ "
-          + StartupOption.HELP.getName() + " ]\n";
   private static final String OM_DAEMON = "om";
   private static boolean securityEnabled = false;
   private OzoneDelegationTokenSecretManager delegationTokenMgr;
@@ -924,109 +916,36 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     return rpcServer;
   }
 
-  /**
-   * Main entry point for starting OzoneManager.
-   *
-   * @param argv arguments
-   * @throws IOException if startup fails due to I/O error
-   */
-  public static void main(String[] argv) throws IOException {
-    if (DFSUtil.parseHelpArgument(argv, USAGE, System.out, true)) {
-      System.exit(0);
-    }
-    try {
-      TracingUtil.initTracing("OzoneManager");
-      OzoneConfiguration conf = new OzoneConfiguration();
-      GenericOptionsParser hParser = new GenericOptionsParser(conf, argv);
-      if (!hParser.isParseSuccessful()) {
-        System.err.println("USAGE: " + USAGE + " \n");
-        hParser.printGenericCommandUsage(System.err);
-        System.exit(1);
-      }
-      OzoneManager om = createOm(hParser.getRemainingArgs(), conf, true);
-      if (om != null) {
-        om.start();
-        om.join();
-      }
-    } catch (Throwable t) {
-      LOG.error("Failed to start the OzoneManager.", t);
-      terminate(1, t);
-    }
-  }
-
-  private static void printUsage(PrintStream out) {
-    out.println(USAGE + "\n");
-  }
-
   private static boolean isOzoneSecurityEnabled() {
     return securityEnabled;
   }
 
   /**
-   * Constructs OM instance based on command line arguments.
-   *
-   * This method is intended for unit tests only. It suppresses the
-   * startup/shutdown message and skips registering Unix signal
-   * handlers.
+   * Constructs OM instance based on the configuration.
    *
-   * @param argv Command line arguments
    * @param conf OzoneConfiguration
    * @return OM instance
    * @throws IOException, AuthenticationException in case OM instance
    *   creation fails.
    */
-  @VisibleForTesting
-  public static OzoneManager createOm(
-      String[] argv, OzoneConfiguration conf)
+  public static OzoneManager createOm(OzoneConfiguration conf)
       throws IOException, AuthenticationException {
-    return createOm(argv, conf, false);
+    loginOMUserIfSecurityEnabled(conf);
+    return new OzoneManager(conf);
   }
 
   /**
-   * Constructs OM instance based on command line arguments.
+   * Logs in the OM use if security is enabled in the configuration.
    *
-   * @param argv Command line arguments
    * @param conf OzoneConfiguration
-   * @param printBanner if true then log a verbose startup message.
-   * @return OM instance
-   * @throws IOException, AuthenticationException in case OM instance
-   *   creation fails.
+   * @throws IOException, AuthenticationException in case login failes.
    */
-  private static OzoneManager createOm(String[] argv,
-      OzoneConfiguration conf, boolean printBanner)
+  private static void loginOMUserIfSecurityEnabled(OzoneConfiguration  conf)
       throws IOException, AuthenticationException {
-    StartupOption startOpt = parseArguments(argv);
-    if (startOpt == null) {
-      printUsage(System.err);
-      terminate(1);
-      return null;
-    }
-
     securityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf);
     if (securityEnabled) {
       loginOMUser(conf);
     }
-
-    switch (startOpt) {
-    case INIT:
-      if (printBanner) {
-        StringUtils.startupShutdownMessage(OzoneManager.class, argv, LOG);
-      }
-      terminate(omInit(conf) ? 0 : 1);
-      return null;
-    case HELP:
-      printUsage(System.err);
-      terminate(0);
-      return null;
-    default:
-      if (argv == null) {
-        argv = new String[]{};
-      }
-      if (printBanner) {
-        StringUtils.startupShutdownMessage(OzoneManager.class, argv, LOG);
-      }
-      return new OzoneManager(conf);
-    }
   }
 
   /**
@@ -1038,7 +957,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    *                     accessible
    */
   @VisibleForTesting
-  public static boolean omInit(OzoneConfiguration conf) throws IOException {
+  public static boolean omInit(OzoneConfiguration conf) throws IOException,
+      AuthenticationException {
+    loginOMUserIfSecurityEnabled(conf);
     OMStorage omStorage = new OMStorage(conf);
     StorageState state = omStorage.getState();
     if (state != StorageState.INITIALIZED) {
@@ -1135,23 +1056,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     }
   }
 
-  /**
-   * Parses the command line options for OM initialization.
-   *
-   * @param args command line arguments
-   * @return StartupOption if options are valid, null otherwise
-   */
-  private static StartupOption parseArguments(String[] args) {
-    if (args == null || args.length == 0) {
-      return StartupOption.REGULAR;
-    } else {
-      if (args.length == 1) {
-        return StartupOption.parse(args[0]);
-      }
-    }
-    return null;
-  }
-
   /**
    * Builds a message for logging startup information about an RPC server.
    *
@@ -3144,34 +3048,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     }
   }
 
-  /**
-   * Startup options.
-   */
-  public enum StartupOption {
-    INIT("--init"),
-    HELP("--help"),
-    REGULAR("--regular");
-
-    private final String name;
-
-    StartupOption(String arg) {
-      this.name = arg;
-    }
-
-    public static StartupOption parse(String value) {
-      for (StartupOption option : StartupOption.values()) {
-        if (option.name.equalsIgnoreCase(value)) {
-          return option;
-        }
-      }
-      return null;
-    }
-
-    public String getName() {
-      return name;
-    }
-  }
-
   public static  Logger getLogger() {
     return LOG;
   }

+ 130 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java

@@ -0,0 +1,130 @@
+/**
+ * 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.om;
+
+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.tracing.TracingUtil;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+
+import java.io.IOException;
+
+/**
+ * This class provides a command line interface to start the OM
+ * using Picocli.
+ */
+@Command(name = "ozone om",
+    hidden = true, description = "Start or initialize the Ozone Manager.",
+    versionProvider = HddsVersionProvider.class,
+    mixinStandardHelpOptions = true)
+public class OzoneManagerStarter extends GenericCli {
+
+  private OzoneConfiguration conf;
+  private OMStarterInterface receiver;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OzoneManagerStarter.class);
+
+  public static void main(String[] args) throws Exception {
+    TracingUtil.initTracing("OzoneManager");
+    new OzoneManagerStarter(
+        new OzoneManagerStarter.OMStarterHelper()).run(args);
+  }
+
+  public OzoneManagerStarter(OMStarterInterface receiverObj) {
+    super();
+    receiver = receiverObj;
+  }
+
+  @Override
+  public Void call() throws Exception {
+    /**
+     * This method is invoked only when a sub-command is not called. Therefore
+     * if someone runs "ozone om" with no parameters, this is the methood
+     * which runs and starts the OM.
+     */
+    commonInit();
+    startOm();
+    return null;
+  }
+
+  /**
+   * This function is used by the command line to start the OM.
+   */
+  private void startOm() throws Exception {
+    receiver.start(conf);
+  }
+
+  /**
+   * This function implements a sub-command to allow the OM to be
+   * initialized from the command line.
+   */
+  @CommandLine.Command(name = "--init",
+      customSynopsis = "ozone om [global options] --init",
+      hidden = false,
+      description = "Initialize the Ozone Manager if not already initialized",
+      mixinStandardHelpOptions = true,
+      versionProvider = HddsVersionProvider.class)
+  public void initOm()
+      throws Exception {
+    commonInit();
+    boolean result = receiver.init(conf);
+    if (!result) {
+      throw new IOException("OM Init failed.");
+    }
+  }
+
+  /**
+   * This function should be called by each command to ensure the configuration
+   * is set and print the startup banner message.
+   */
+  private void commonInit() {
+    conf = createOzoneConfiguration();
+
+    String[] originalArgs = getCmd().getParseResult().originalArgs()
+        .toArray(new String[0]);
+    StringUtils.startupShutdownMessage(OzoneManager.class,
+        originalArgs, LOG);
+  }
+
+  /**
+   * This static class wraps the external dependencies needed for this command
+   * to execute its tasks. This allows the dependency to be injected for unit
+   * testing.
+   */
+  static class OMStarterHelper implements OMStarterInterface{
+
+    public void start(OzoneConfiguration conf) throws IOException,
+        AuthenticationException {
+      OzoneManager om = OzoneManager.createOm(conf);
+      om.start();
+      om.join();
+    }
+
+    public boolean init(OzoneConfiguration conf) throws IOException,
+        AuthenticationException {
+      return OzoneManager.omInit(conf);
+    }
+  }
+
+}

+ 154 - 0
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java

@@ -0,0 +1,154 @@
+/**
+ * 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.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import static org.junit.Assert.*;
+
+/**
+ * This class is used to test the CLI provided by OzoneManagerStarter, which is
+ * used to start and init the OzoneManager. The calls to the Ozone Manager are
+ * mocked so the tests only validate the CLI calls the correct methods are
+ * invoked.
+ */
+public class TestOzoneManagerStarter {
+
+  private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private final ByteArrayOutputStream errContent = new ByteArrayOutputStream();
+  private final PrintStream originalOut = System.out;
+  private final PrintStream originalErr = System.err;
+
+  private MockOMStarter mock;
+
+  @Before
+  public void setUpStreams() {
+    System.setOut(new PrintStream(outContent));
+    System.setErr(new PrintStream(errContent));
+    mock = new MockOMStarter();
+  }
+
+  @After
+  public void restoreStreams() {
+    System.setOut(originalOut);
+    System.setErr(originalErr);
+  }
+
+  @Test
+  public void testCallsStartWhenServerStarted() throws Exception {
+    executeCommand();
+    assertTrue(mock.startCalled);
+  }
+
+  @Test
+  public void testExceptionThrownWhenStartFails() throws Exception {
+    mock.throwOnStart = true;
+    try {
+      executeCommand();
+      fail("Exception should have been thrown");
+    } catch (Exception e) {
+      assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testStartNotCalledWithInvalidParam() throws Exception {
+    executeCommand("--invalid");
+    assertFalse(mock.startCalled);
+  }
+
+  @Test
+  public void testPassingInitSwitchCallsInit() {
+    executeCommand("--init");
+    assertTrue(mock.initCalled);
+  }
+
+  @Test
+  public void testInitSwitchWithInvalidParamDoesNotRun() {
+    executeCommand("--init", "--invalid");
+    assertFalse(mock.initCalled);
+  }
+
+  @Test
+  public void testUnSuccessfulInitThrowsException() {
+    mock.throwOnInit = true;
+    try {
+      executeCommand("--init");
+      fail("Exception show have been thrown");
+    } catch (Exception e) {
+      assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testInitThatReturnsFalseThrowsException() {
+    mock.initStatus = false;
+    try {
+      executeCommand("--init");
+      fail("Exception show have been thrown");
+    } catch (Exception e) {
+      assertTrue(true);
+    }
+  }
+
+  @Test
+  public void testUsagePrintedOnInvalidInput() {
+    executeCommand("--invalid");
+    Pattern p = Pattern.compile("^Unknown option:.*--invalid.*\nUsage");
+    Matcher m = p.matcher(errContent.toString());
+    assertTrue(m.find());
+  }
+
+  private void executeCommand(String... args) {
+    new OzoneManagerStarter(mock).execute(args);
+  }
+
+  static class MockOMStarter implements OMStarterInterface {
+
+    private boolean startCalled = false;
+    private boolean initCalled = false;
+    private boolean initStatus = true;
+    private boolean throwOnStart = false;
+    private boolean throwOnInit = false;
+
+    public void start(OzoneConfiguration conf) throws IOException,
+        AuthenticationException {
+      startCalled = true;
+      if (throwOnStart) {
+        throw new IOException("Simulated Exception");
+      }
+    }
+
+    public boolean init(OzoneConfiguration conf) throws IOException,
+        AuthenticationException {
+      initCalled = true;
+      if (throwOnInit) {
+        throw new IOException("Simulated Exception");
+      }
+      return initStatus;
+    }
+  }
+}

+ 1 - 1
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java

@@ -186,7 +186,7 @@ public final class GenesisUtil {
       omStorage.setOmId(UUID.randomUUID().toString());
       omStorage.initialize();
     }
-    return OzoneManager.createOm(null, conf);
+    return OzoneManager.createOm(conf);
   }
 
   static void configureOM(Configuration conf, int numHandlers) {