+ * 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.
+ */
+
+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;
+}
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 4041670eaa9ce..287c2ded96055 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -142,12 +142,10 @@
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.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.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 @@ private RPC.Server startRpcServer(OzoneConfiguration conf,
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 @@ private static OzoneManager createOm(String[] argv,
* 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 @@ private static ScmInfo getScmInfo(OzoneConfiguration conf)
}
}
- /**
- * 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 List
+ * 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.
+ */
+
+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);
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManagerStarter.java
new file mode 100644
index 0000000000000..80281693c1b62
--- /dev/null
+++ b/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
+ *
+ * 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.
+ */
+
+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;
+ }
+ }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
index 6bff82b5755dc..d08d26ad28938 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/GenesisUtil.java
@@ -186,7 +186,7 @@ static OzoneManager getOm(OzoneConfiguration conf)
omStorage.setOmId(UUID.randomUUID().toString());
omStorage.initialize();
}
- return OzoneManager.createOm(null, conf);
+ return OzoneManager.createOm(conf);
}
static void configureOM(Configuration conf, int numHandlers) {